From f5d7f78e52b42502fcf433f229b7043e618cc39f Mon Sep 17 00:00:00 2001 From: Prajithp Date: Thu, 26 Sep 2024 13:38:46 +0530 Subject: [PATCH 01/18] add scaler for temporal Signed-off-by: Prajithp --- pkg/scalers/temporal.go | 181 ++++++++++++++++ pkg/scalers/temporal_test.go | 139 +++++++++++++ pkg/scaling/scalers_builder.go | 2 + tests/scalers/temporal/temporal_test.go | 265 ++++++++++++++++++++++++ 4 files changed, 587 insertions(+) create mode 100644 pkg/scalers/temporal.go create mode 100644 pkg/scalers/temporal_test.go create mode 100644 tests/scalers/temporal/temporal_test.go diff --git a/pkg/scalers/temporal.go b/pkg/scalers/temporal.go new file mode 100644 index 00000000000..43c656bb519 --- /dev/null +++ b/pkg/scalers/temporal.go @@ -0,0 +1,181 @@ +package scalers + +import ( + "context" + "errors" + "fmt" + "log/slog" + "strconv" + "time" + + "github.com/go-logr/logr" + "github.com/kedacore/keda/v2/pkg/scalers/scalersconfig" + kedautil "github.com/kedacore/keda/v2/pkg/util" + sdk "go.temporal.io/sdk/client" + sdklog "go.temporal.io/sdk/log" + "google.golang.org/grpc" + v2 "k8s.io/api/autoscaling/v2" + "k8s.io/metrics/pkg/apis/external_metrics" +) + +const ( + temporalDefaultTargetQueueLength = 5 + temporalDefaultActivationQueueLength = 0 + temporalDefaultNamespace = "default" +) + +type temporalScaler struct { + metricType v2.MetricTargetType + metadata *temporalMetadata + tcl sdk.Client + logger logr.Logger +} + +type temporalMetadata struct { + activationLagThreshold int64 + endpoint string + namespace string + triggerIndex int + targetQueueSize int64 + queueName string +} + +func NewTemporalScaler(config *scalersconfig.ScalerConfig) (Scaler, error) { + metricType, err := GetMetricTargetType(config) + if err != nil { + return nil, fmt.Errorf("failed to get scaler metric type: %w", err) + } + + meta, err := parseTemporalMetadata(config) + if err != nil { + return nil, fmt.Errorf("failed to parse Temporal metadata: %w", err) + } + + logger := InitializeLogger(config, "temporal_scaler") + + c, err := getTemporalClient(meta) + if err != nil { + return nil, fmt.Errorf("failed to create Temporal client connection: %w", err) + } + + return &temporalScaler{ + metricType: metricType, + metadata: meta, + tcl: c, + logger: logger, + }, nil +} + +func (s *temporalScaler) Close(_ context.Context) error { + if s.tcl != nil { + s.tcl.Close() + } + return nil +} + +func (s *temporalScaler) GetMetricSpecForScaling(context.Context) []v2.MetricSpec { + metricName := kedautil.NormalizeString(fmt.Sprintf("temporal-%s-%s", s.metadata.namespace, s.metadata.queueName)) + externalMetric := &v2.ExternalMetricSource{ + Metric: v2.MetricIdentifier{ + Name: GenerateMetricNameWithIndex(s.metadata.triggerIndex, metricName), + }, + Target: GetMetricTarget(s.metricType, s.metadata.targetQueueSize), + } + metricSpec := v2.MetricSpec{ + External: externalMetric, + Type: externalMetricType, + } + return []v2.MetricSpec{metricSpec} +} + +func (s *temporalScaler) GetMetricsAndActivity(ctx context.Context, metricName string) ([]external_metrics.ExternalMetricValue, bool, error) { + queueSize, err := s.getQueueSize(ctx) + if err != nil { + return nil, false, fmt.Errorf("failed to get Temporal queue size: %w", err) + } + + metric := GenerateMetricInMili(metricName, float64(queueSize)) + + return []external_metrics.ExternalMetricValue{metric}, queueSize > s.metadata.activationLagThreshold, nil +} + +func (s *temporalScaler) getQueueSize(ctx context.Context) (int64, error) { + resp, err := s.tcl.DescribeTaskQueueEnhanced(ctx, sdk.DescribeTaskQueueEnhancedOptions{ + TaskQueue: s.metadata.queueName, + ReportStats: true, + }) + if err != nil { + return 0, fmt.Errorf("failed to get Temporal queue size: %w", err) + } + + return getCombinedBacklogCount(resp), nil +} + +func getCombinedBacklogCount(description sdk.TaskQueueDescription) int64 { + var count int64 + for _, versionInfo := range description.VersionsInfo { + for _, typeInfo := range versionInfo.TypesInfo { + if typeInfo.Stats != nil { + count += typeInfo.Stats.ApproximateBacklogCount + } + } + } + return count +} + +func getTemporalClient(meta *temporalMetadata) (sdk.Client, error) { + return sdk.Dial(sdk.Options{ + HostPort: meta.endpoint, + Namespace: meta.namespace, + Logger: sdklog.NewStructuredLogger(slog.Default()), + ConnectionOptions: sdk.ConnectionOptions{ + DialOptions: []grpc.DialOption{ + grpc.WithConnectParams(grpc.ConnectParams{ + MinConnectTimeout: 5 * time.Second, + }), + }, + }, + }) +} + +func parseTemporalMetadata(config *scalersconfig.ScalerConfig) (*temporalMetadata, error) { + meta := &temporalMetadata{} + meta.activationLagThreshold = temporalDefaultActivationQueueLength + meta.targetQueueSize = temporalDefaultTargetQueueLength + + if config.TriggerMetadata["endpoint"] == "" { + return nil, errors.New("no Temporal gRPC endpoint provided") + } + meta.endpoint = config.TriggerMetadata["endpoint"] + + if config.TriggerMetadata["namespace"] == "" { + meta.namespace = temporalDefaultNamespace + } else { + meta.namespace = config.TriggerMetadata["namespace"] + } + + if size, ok := config.TriggerMetadata["targetQueueSize"]; ok { + queueSize, err := strconv.ParseInt(size, 10, 64) + if err != nil { + return nil, fmt.Errorf("invalid targetQueueSize - must be an integer") + } + meta.targetQueueSize = queueSize + } + + if size, ok := config.TriggerMetadata["activationTargetQueueSize"]; ok { + activationTargetQueueSize, err := strconv.ParseInt(size, 10, 64) + if err != nil { + return nil, fmt.Errorf("invalid activationTargetQueueSize - must be an integer") + } + meta.activationLagThreshold = activationTargetQueueSize + } + + if queueName, ok := config.TriggerMetadata["queueName"]; ok { + meta.queueName = queueName + } else { + return nil, errors.New("no queueName provided") + } + + meta.triggerIndex = config.TriggerIndex + return meta, nil +} diff --git a/pkg/scalers/temporal_test.go b/pkg/scalers/temporal_test.go new file mode 100644 index 00000000000..d95fc5e4176 --- /dev/null +++ b/pkg/scalers/temporal_test.go @@ -0,0 +1,139 @@ +package scalers + +import ( + "context" + "testing" + + "github.com/kedacore/keda/v2/pkg/scalers/scalersconfig" + "github.com/stretchr/testify/assert" +) + +var ( + temporalEndpoint = "localhost:7233" + temporalNamespace = "v2" + temporalQueueName = "default" +) + +type parseTemporalMetadataTestData struct { + metadata map[string]string + isError bool +} + +type temporalMetricIdentifier struct { + metadataTestData *parseTemporalMetadataTestData + triggerIndex int + name string +} + +var testTemporalMetadata = []parseTemporalMetadataTestData{ + // nothing passed + {map[string]string{}, true}, + // Missing queueName, should fail + {map[string]string{"endpoint": temporalEndpoint, "namespace": temporalNamespace}, true}, + // Missing namespace, should success + {map[string]string{"endpoint": temporalEndpoint, "queueName": temporalQueueName}, false}, + // Missing endpoint, should fail + {map[string]string{"queueName": temporalQueueName, "namespace": temporalNamespace}, true}, + // All good. + {map[string]string{"endpoint": temporalEndpoint, "queueName": temporalQueueName, "namespace": temporalNamespace}, false}, + // All good + activationLagThreshold + {map[string]string{"endpoint": temporalEndpoint, "queueName": temporalQueueName, "namespace": temporalNamespace, "activationTargetQueueSize": "10"}, false}, +} + +var temporalMetricIdentifiers = []temporalMetricIdentifier{ + {&testTemporalMetadata[4], 0, "s0-temporal-v2-default"}, + {&testTemporalMetadata[4], 1, "s1-temporal-v2-default"}, +} + +func TestTemporalGetMetricSpecForScaling(t *testing.T) { + for _, testData := range temporalMetricIdentifiers { + ctx := context.Background() + meta, err := parseTemporalMetadata(&scalersconfig.ScalerConfig{ + TriggerMetadata: testData.metadataTestData.metadata, + TriggerIndex: testData.triggerIndex, + }) + if err != nil { + t.Fatal("Could not parse metadata:", err) + } + mockTemporalScaler := temporalScaler{ + metadata: meta, + } + + metricSpec := mockTemporalScaler.GetMetricSpecForScaling(ctx) + metricName := metricSpec[0].External.Metric.Name + if metricName != testData.name { + t.Error("Wrong External metric source name:", metricName) + } + } +} + +func TestParseTemporalMetadata(t *testing.T) { + cases := []struct { + name string + metadata map[string]string + wantMeta *temporalMetadata + wantErr bool + }{ + { + name: "empty metadata", + wantMeta: nil, + wantErr: true, + }, + { + name: "empty queue name", + metadata: map[string]string{ + "endpoint": "test:7233", + "namespace": "default", + }, + wantMeta: nil, + wantErr: true, + }, + { + name: "empty namespace", + metadata: map[string]string{ + "endpoint": "test:7233", + "queueName": "testxx", + }, + wantMeta: &temporalMetadata{ + endpoint: "test:7233", + namespace: "default", + queueName: "testxx", + targetQueueSize: 5, + }, + wantErr: false, + }, + { + name: "activationTargetQueueSize should not be 0", + metadata: map[string]string{ + "endpoint": "test:7233", + "namespace": "default", + "queueName": "testxx", + "activationTargetQueueSize": "12", + }, + wantMeta: &temporalMetadata{ + endpoint: "test:7233", + namespace: "default", + queueName: "testxx", + targetQueueSize: 5, + activationLagThreshold: 12, + }, + wantErr: false, + }, + } + + for _, testCase := range cases { + c := testCase + t.Run(c.name, func(t *testing.T) { + config := &scalersconfig.ScalerConfig{ + TriggerMetadata: c.metadata, + } + meta, err := parseTemporalMetadata(config) + if c.wantErr == true && err != nil { + t.Log("Expected error, got err") + } else { + assert.NoError(t, err) + } + assert.Equal(t, c.wantMeta, meta) + }) + } +} diff --git a/pkg/scaling/scalers_builder.go b/pkg/scaling/scalers_builder.go index 1f4549c7ffa..f29b313b210 100644 --- a/pkg/scaling/scalers_builder.go +++ b/pkg/scaling/scalers_builder.go @@ -251,6 +251,8 @@ func buildScaler(ctx context.Context, client client.Client, triggerType string, return scalers.NewSplunkScaler(config) case "stan": return scalers.NewStanScaler(config) + case "temporal": + return scalers.NewTemporalScaler(config) default: return nil, fmt.Errorf("no scaler found for type: %s", triggerType) } diff --git a/tests/scalers/temporal/temporal_test.go b/tests/scalers/temporal/temporal_test.go new file mode 100644 index 00000000000..54c281d8aa1 --- /dev/null +++ b/tests/scalers/temporal/temporal_test.go @@ -0,0 +1,265 @@ +//go:build e2e +// +build e2e + +package temporal_test + +import ( + "fmt" + "testing" + + "github.com/joho/godotenv" + . "github.com/kedacore/keda/v2/tests/helper" + "github.com/stretchr/testify/assert" + "k8s.io/client-go/kubernetes" +) + +var _ = godotenv.Load("../../.env") + +const ( + testName = "temporal-test" +) + +var ( + testNamespace = fmt.Sprintf("%s-ns", testName) + + TemporalDeploymentName = fmt.Sprintf("temporal-%s", testName) + + scaledObjectName = fmt.Sprintf("%s-so", testName) + deploymentName = fmt.Sprintf("%s", testName) +) + +type templateData struct { + DeploymentName string + TestNamespace string + TemporalDeploymentName string + ScaledObjectName string +} + +const ( + temporalServiceTemplate = ` +apiVersion: v1 +kind: Service +metadata: + name: {{.TemporalDeploymentName}} + namespace: {{.TestNamespace}} +spec: + type: ClusterIP + ports: + - port: 7233 + protocol: TCP + targetPort: 7233 + selector: + app: temporal +` + + temporalDeploymentTemplate = ` +apiVersion: apps/v1 +kind: Deployment +metadata: + name: {{.TemporalDeploymentName}} + namespace: {{.TestNamespace}} + labels: + app: temporal +spec: + replicas: 1 + selector: + matchLabels: + app: temporal + template: + metadata: + labels: + app: temporal + spec: + containers: + - name: ubuntu + image: ubuntu:latest + command: ["bash", "-c"] + args: + - | + apt-get update && apt-get install -y curl && \ + curl -sSf https://temporal.download/cli.sh | sh && \ + PATH="\$PATH:/root/.temporalio/bin" >> ~/.bashrc && \ + source ~/.bashrc && \ + temporal server start-dev --namespace v2 --ip 0.0.0.0 + ports: + - containerPort: 7233 + livenessProbe: + exec: + command: + - /bin/sh + - -ec + - test $(ps -ef | grep -v grep | grep "temporal server" | wc -l) -eq 1 + failureThreshold: 5 + initialDelaySeconds: 10 + periodSeconds: 30 + successThreshold: 1 + timeoutSeconds: 1 + readinessProbe: + exec: + command: + - /bin/sh + - -ec + - test $(ps -ef | grep -v grep | grep "temporal server" | wc -l) -eq 1 + failureThreshold: 5 + initialDelaySeconds: 10 + periodSeconds: 30 + successThreshold: 1 + timeoutSeconds: 1 +` + + scaledObjectTemplate = ` +apiVersion: keda.sh/v1alpha1 +kind: ScaledObject +metadata: + name: {{.ScaledObjectName}} + namespace: {{.TestNamespace}} +spec: + scaleTargetRef: + name: {{.DeploymentName}} + pollingInterval: 5 + cooldownPeriod: 10 + minReplicaCount: 0 + maxReplicaCount: 1 + advanced: + horizontalPodAutoscalerConfig: + behavior: + scaleDown: + stabilizationWindowSeconds: 10 + triggers: + - type: temporal + metadata: + namespace: default + queueName: hello-task-queue + targetQueueSize: "2" + activationTargetQueueSize: "3" + endpoint: {{.TemporalDeploymentName}}.{{.TestNamespace}}.svc.cluster.local:7233 +` + + deploymentTemplate = ` +apiVersion: apps/v1 +kind: Deployment +metadata: + name: {{.DeploymentName}} + namespace: {{.TestNamespace}} + labels: + app: {{.DeploymentName}} +spec: + replicas: 0 + selector: + matchLabels: + app: {{.DeploymentName}} + template: + metadata: + labels: + app: {{.DeploymentName}} + spec: + containers: + - name: nginx + image: nginxinc/nginx-unprivileged + ports: + - containerPort: 80 +` + + jobWorkerTemplate = ` +apiVersion: batch/v1 +kind: Job +metadata: + name: worker + namespace: {{.TestNamespace}} +spec: + template: + spec: + containers: + - name: worker + image: "prajithp/temporal-sample:1.0.0" + imagePullPolicy: Always + env: + - name: TEMPORAL_ADDR + value: {{.TemporalDeploymentName}}.{{.TestNamespace}}.svc.cluster.local:7233 + - name: TEMPORAL_NAMESPACE + value: default + - name: MODE + value: WORKER + restartPolicy: OnFailure + backoffLimit: 4 +` + + jobWorkeFlowTemplate = ` +apiVersion: batch/v1 +kind: Job +metadata: + name: workerflow + namespace: {{.TestNamespace}} +spec: + template: + spec: + containers: + - name: workerflow + image: "prajithp/temporal-sample:1.0.0" + imagePullPolicy: Always + env: + - name: TEMPORAL_ADDR + value: {{.TemporalDeploymentName}}.{{.TestNamespace}}.svc.cluster.local:7233 + restartPolicy: OnFailure + backoffLimit: 4 +` +) + +func getTemplateData() (templateData, []Template) { + return templateData{ + TestNamespace: testNamespace, + TemporalDeploymentName: TemporalDeploymentName, + ScaledObjectName: scaledObjectName, + DeploymentName: deploymentName, + }, []Template{ + {Name: "scaledObjectTemplate", Config: scaledObjectTemplate}, + {Name: "deploymentTemplate", Config: deploymentTemplate}, + } +} + +func TestTemporalScaler(t *testing.T) { + kc := GetKubernetesClient(t) + data, templates := getTemplateData() + CreateNamespace(t, kc, testNamespace) + + KubectlApplyWithTemplate(t, data, "temporalServiceTemplate", temporalServiceTemplate) + KubectlApplyWithTemplate(t, data, "temporalDeploymentTemplate", temporalDeploymentTemplate) + assert.True(t, WaitForDeploymentReplicaReadyCount(t, kc, TemporalDeploymentName, testNamespace, 1, 30, 4), "temporal is not in a ready state") + + KubectlApplyMultipleWithTemplate(t, data, templates) + testActivation(t, kc, data) + testScaleOut(t, kc, data) + testScaleIn(t, kc, data) + DeleteKubernetesResources(t, testNamespace, data, templates) +} + +func testActivation(t *testing.T, kc *kubernetes.Clientset, data templateData) { + t.Log("--- testing activation ---") + + KubectlApplyWithTemplate(t, data, "jobWorkFlowActivation", jobWorkeFlowTemplate) + assert.True(t, WaitForJobCount(t, kc, testNamespace, 1, 60, 3), "job count in namespace should be 1") + + AssertReplicaCountNotChangeDuringTimePeriod(t, kc, deploymentName, testNamespace, 0, 180) + KubectlDeleteWithTemplate(t, data, "jobWorkFlowActivation", jobWorkeFlowTemplate) +} + +func testScaleOut(t *testing.T, kc *kubernetes.Clientset, data templateData) { + t.Log("--- testing scale out ---") + + KubectlApplyWithTemplate(t, data, "jobWorkFlow", jobWorkeFlowTemplate) + assert.True(t, WaitForJobCount(t, kc, testNamespace, 1, 60, 3), "job count in namespace should be 1") + + assert.True(t, WaitForDeploymentReplicaReadyCount(t, kc, deploymentName, testNamespace, 1, 60, 3), + "replica count should be %d after 3 minutes", 1) +} + +func testScaleIn(t *testing.T, kc *kubernetes.Clientset, data templateData) { + t.Log("--- testing scale in ---") + + KubectlApplyWithTemplate(t, data, "jobWorker", jobWorkerTemplate) + // workflow is already waiting for response from worker + assert.True(t, WaitForJobCount(t, kc, testNamespace, 2, 60, 3), "job count in namespace should be 2") + + assert.True(t, WaitForDeploymentReplicaReadyCount(t, kc, deploymentName, testNamespace, 0, 60, 5), + "replica count should be %d after 5 minutes", 0) +} From 2cfaa31d3a7713c11520d483d5f967a3aa1aaec9 Mon Sep 17 00:00:00 2001 From: Prajithp Date: Fri, 27 Sep 2024 11:42:32 +0530 Subject: [PATCH 02/18] add option to filter based on build ids Signed-off-by: Prajithp --- pkg/scalers/temporal.go | 93 +++++++++++++++++++++++++++++++++--- pkg/scalers/temporal_test.go | 7 ++- 2 files changed, 92 insertions(+), 8 deletions(-) diff --git a/pkg/scalers/temporal.go b/pkg/scalers/temporal.go index 43c656bb519..af047911a3d 100644 --- a/pkg/scalers/temporal.go +++ b/pkg/scalers/temporal.go @@ -6,6 +6,7 @@ import ( "fmt" "log/slog" "strconv" + "strings" "time" "github.com/go-logr/logr" @@ -22,6 +23,16 @@ const ( temporalDefaultTargetQueueLength = 5 temporalDefaultActivationQueueLength = 0 temporalDefaultNamespace = "default" + temporalDefaultSelectAllActive = true + temporalDefaultSelectUnversioned = true +) + +var ( + temporalDefauleQueueTypes = []sdk.TaskQueueType{ + sdk.TaskQueueTypeActivity, + sdk.TaskQueueTypeWorkflow, + sdk.TaskQueueTypeNexus, + } ) type temporalScaler struct { @@ -38,21 +49,26 @@ type temporalMetadata struct { triggerIndex int targetQueueSize int64 queueName string + queueTypes []string + buildIDs []string + allActive bool + unversioned bool + apiKey *string } func NewTemporalScaler(config *scalersconfig.ScalerConfig) (Scaler, error) { + logger := InitializeLogger(config, "temporal_scaler") + metricType, err := GetMetricTargetType(config) if err != nil { return nil, fmt.Errorf("failed to get scaler metric type: %w", err) } - meta, err := parseTemporalMetadata(config) + meta, err := parseTemporalMetadata(config, logger) if err != nil { return nil, fmt.Errorf("failed to parse Temporal metadata: %w", err) } - logger := InitializeLogger(config, "temporal_scaler") - c, err := getTemporalClient(meta) if err != nil { return nil, fmt.Errorf("failed to create Temporal client connection: %w", err) @@ -100,9 +116,22 @@ func (s *temporalScaler) GetMetricsAndActivity(ctx context.Context, metricName s } func (s *temporalScaler) getQueueSize(ctx context.Context) (int64, error) { + queueType := getQueueTypes(s.metadata.queueTypes) + + var selection *sdk.TaskQueueVersionSelection + if s.metadata.allActive || s.metadata.unversioned || len(s.metadata.buildIDs) > 0 { + selection = &sdk.TaskQueueVersionSelection{ + AllActive: s.metadata.allActive, + Unversioned: s.metadata.unversioned, + BuildIDs: s.metadata.buildIDs, + } + } + resp, err := s.tcl.DescribeTaskQueueEnhanced(ctx, sdk.DescribeTaskQueueEnhancedOptions{ - TaskQueue: s.metadata.queueName, - ReportStats: true, + TaskQueue: s.metadata.queueName, + ReportStats: true, + Versions: selection, + TaskQueueTypes: queueType, }) if err != nil { return 0, fmt.Errorf("failed to get Temporal queue size: %w", err) @@ -111,6 +140,27 @@ func (s *temporalScaler) getQueueSize(ctx context.Context) (int64, error) { return getCombinedBacklogCount(resp), nil } +func getQueueTypes(queueTypes []string) []sdk.TaskQueueType { + var taskQueueTypes []sdk.TaskQueueType + for _, t := range queueTypes { + var taskQueueType sdk.TaskQueueType + switch t { + case "workflow": + taskQueueType = sdk.TaskQueueTypeWorkflow + case "activity": + taskQueueType = sdk.TaskQueueTypeActivity + case "nexus": + taskQueueType = sdk.TaskQueueTypeNexus + } + taskQueueTypes = append(taskQueueTypes, taskQueueType) + } + + if len(taskQueueTypes) == 0 { + return temporalDefauleQueueTypes + } + return taskQueueTypes +} + func getCombinedBacklogCount(description sdk.TaskQueueDescription) int64 { var count int64 for _, versionInfo := range description.VersionsInfo { @@ -138,7 +188,7 @@ func getTemporalClient(meta *temporalMetadata) (sdk.Client, error) { }) } -func parseTemporalMetadata(config *scalersconfig.ScalerConfig) (*temporalMetadata, error) { +func parseTemporalMetadata(config *scalersconfig.ScalerConfig, logger logr.Logger) (*temporalMetadata, error) { meta := &temporalMetadata{} meta.activationLagThreshold = temporalDefaultActivationQueueLength meta.targetQueueSize = temporalDefaultTargetQueueLength @@ -176,6 +226,37 @@ func parseTemporalMetadata(config *scalersconfig.ScalerConfig) (*temporalMetadat return nil, errors.New("no queueName provided") } + // if buildIds is provided, it will be used to filter the queue and make sure + // selectAllActive and selectUnversioned are set to false to avoid considering + if buildIds, ok := config.TriggerMetadata["buildIds"]; ok && buildIds != "" { + meta.buildIDs = strings.Split(buildIds, ",") + } + + if val, ok := config.TriggerMetadata["selectAllActive"]; ok && val != "" { + allActive, err := strconv.ParseBool(val) + if err != nil { + meta.allActive = temporalDefaultSelectAllActive + logger.Error(err, "Error parsing Temoral queue metadata selectAllActive, using default %n", temporalDefaultSelectAllActive) + } else { + meta.allActive = allActive + } + } + + if val, ok := config.TriggerMetadata["selectUnversioned"]; ok && val != "" { + unversioned, err := strconv.ParseBool(val) + if err != nil { + meta.unversioned = temporalDefaultSelectUnversioned + logger.Error(err, "Error parsing Temoral queue metadata selectUnversioned, using default %n", temporalDefaultSelectUnversioned) + } else { + meta.unversioned = unversioned + } + } + + // optional, valide queueTypes are workflow, activity, nexus + if val, ok := config.TriggerMetadata["queueTypes"]; ok && val != "" { + meta.queueTypes = strings.Split(val, ",") + } + meta.triggerIndex = config.TriggerIndex return meta, nil } diff --git a/pkg/scalers/temporal_test.go b/pkg/scalers/temporal_test.go index d95fc5e4176..e3c9ee70524 100644 --- a/pkg/scalers/temporal_test.go +++ b/pkg/scalers/temporal_test.go @@ -4,6 +4,7 @@ import ( "context" "testing" + "github.com/go-logr/logr" "github.com/kedacore/keda/v2/pkg/scalers/scalersconfig" "github.com/stretchr/testify/assert" ) @@ -12,6 +13,8 @@ var ( temporalEndpoint = "localhost:7233" temporalNamespace = "v2" temporalQueueName = "default" + + logger = logr.Discard() ) type parseTemporalMetadataTestData struct { @@ -51,7 +54,7 @@ func TestTemporalGetMetricSpecForScaling(t *testing.T) { meta, err := parseTemporalMetadata(&scalersconfig.ScalerConfig{ TriggerMetadata: testData.metadataTestData.metadata, TriggerIndex: testData.triggerIndex, - }) + }, logger) if err != nil { t.Fatal("Could not parse metadata:", err) } @@ -127,7 +130,7 @@ func TestParseTemporalMetadata(t *testing.T) { config := &scalersconfig.ScalerConfig{ TriggerMetadata: c.metadata, } - meta, err := parseTemporalMetadata(config) + meta, err := parseTemporalMetadata(config, logger) if c.wantErr == true && err != nil { t.Log("Expected error, got err") } else { From 6018463586628be8c15b6fd4ee07fcd82526a6bf Mon Sep 17 00:00:00 2001 From: Prajithp Date: Fri, 27 Sep 2024 22:00:57 +0530 Subject: [PATCH 03/18] use typed config Signed-off-by: Prajithp --- pkg/scalers/temporal.go | 143 ++++++++++------------------------- pkg/scalers/temporal_test.go | 135 ++++++++++++++++++++++++++------- 2 files changed, 150 insertions(+), 128 deletions(-) diff --git a/pkg/scalers/temporal.go b/pkg/scalers/temporal.go index af047911a3d..f1b5eb476c2 100644 --- a/pkg/scalers/temporal.go +++ b/pkg/scalers/temporal.go @@ -2,11 +2,8 @@ package scalers import ( "context" - "errors" "fmt" "log/slog" - "strconv" - "strings" "time" "github.com/go-logr/logr" @@ -19,14 +16,6 @@ import ( "k8s.io/metrics/pkg/apis/external_metrics" ) -const ( - temporalDefaultTargetQueueLength = 5 - temporalDefaultActivationQueueLength = 0 - temporalDefaultNamespace = "default" - temporalDefaultSelectAllActive = true - temporalDefaultSelectUnversioned = true -) - var ( temporalDefauleQueueTypes = []sdk.TaskQueueType{ sdk.TaskQueueTypeActivity, @@ -43,17 +32,29 @@ type temporalScaler struct { } type temporalMetadata struct { - activationLagThreshold int64 - endpoint string - namespace string - triggerIndex int - targetQueueSize int64 - queueName string - queueTypes []string - buildIDs []string - allActive bool - unversioned bool - apiKey *string + ActivationLagThreshold int64 `keda:"name=activationTargetQueueSize, order=triggerMetadata, default=0"` + Endpoint string `keda:"name=endpoint, order=triggerMetadata;resolvedEnv"` + Namespace string `keda:"name=namespace, order=triggerMetadata, default=default"` + TargetQueueSize int64 `keda:"name=targetQueueSize, order=triggerMetadata, default=5"` + QueueName string `keda:"name=queueName, order=triggerMetadata"` + QueueTypes []string `keda:"name=queueTypes, order=triggerMetadata, optional"` + BuildIDs []string `keda:"name=buildIds, order=triggerMetadata, optional"` + AllActive bool `keda:"name=selectAllActive, order=triggerMetadata, default=true"` + Unversioned bool `keda:"name=selectUnversioned, order=triggerMetadata, default=true"` + ApiKey string `keda:"name=apiKey, order=authParams;triggerMetadata, optional"` + + triggerIndex int +} + +func (a *temporalMetadata) Validate() error { + if a.TargetQueueSize <= 0 { + return fmt.Errorf("targetQueueSize must be a positive number") + } + if a.ActivationLagThreshold < 0 { + return fmt.Errorf("activationTargetQueueSize must be a positive number") + } + + return nil } func NewTemporalScaler(config *scalersconfig.ScalerConfig) (Scaler, error) { @@ -90,17 +91,19 @@ func (s *temporalScaler) Close(_ context.Context) error { } func (s *temporalScaler) GetMetricSpecForScaling(context.Context) []v2.MetricSpec { - metricName := kedautil.NormalizeString(fmt.Sprintf("temporal-%s-%s", s.metadata.namespace, s.metadata.queueName)) + metricName := kedautil.NormalizeString(fmt.Sprintf("temporal-%s-%s", s.metadata.Namespace, s.metadata.QueueName)) externalMetric := &v2.ExternalMetricSource{ Metric: v2.MetricIdentifier{ Name: GenerateMetricNameWithIndex(s.metadata.triggerIndex, metricName), }, - Target: GetMetricTarget(s.metricType, s.metadata.targetQueueSize), + Target: GetMetricTarget(s.metricType, s.metadata.TargetQueueSize), } + metricSpec := v2.MetricSpec{ External: externalMetric, Type: externalMetricType, } + return []v2.MetricSpec{metricSpec} } @@ -112,23 +115,23 @@ func (s *temporalScaler) GetMetricsAndActivity(ctx context.Context, metricName s metric := GenerateMetricInMili(metricName, float64(queueSize)) - return []external_metrics.ExternalMetricValue{metric}, queueSize > s.metadata.activationLagThreshold, nil + return []external_metrics.ExternalMetricValue{metric}, queueSize > s.metadata.ActivationLagThreshold, nil } func (s *temporalScaler) getQueueSize(ctx context.Context) (int64, error) { - queueType := getQueueTypes(s.metadata.queueTypes) - var selection *sdk.TaskQueueVersionSelection - if s.metadata.allActive || s.metadata.unversioned || len(s.metadata.buildIDs) > 0 { + if s.metadata.AllActive || s.metadata.Unversioned || len(s.metadata.BuildIDs) > 0 { selection = &sdk.TaskQueueVersionSelection{ - AllActive: s.metadata.allActive, - Unversioned: s.metadata.unversioned, - BuildIDs: s.metadata.buildIDs, + AllActive: s.metadata.AllActive, + Unversioned: s.metadata.Unversioned, + BuildIDs: s.metadata.BuildIDs, } } + queueType := getQueueTypes(s.metadata.QueueTypes) + resp, err := s.tcl.DescribeTaskQueueEnhanced(ctx, sdk.DescribeTaskQueueEnhancedOptions{ - TaskQueue: s.metadata.queueName, + TaskQueue: s.metadata.QueueName, ReportStats: true, Versions: selection, TaskQueueTypes: queueType, @@ -175,8 +178,8 @@ func getCombinedBacklogCount(description sdk.TaskQueueDescription) int64 { func getTemporalClient(meta *temporalMetadata) (sdk.Client, error) { return sdk.Dial(sdk.Options{ - HostPort: meta.endpoint, - Namespace: meta.namespace, + HostPort: meta.Endpoint, + Namespace: meta.Namespace, Logger: sdklog.NewStructuredLogger(slog.Default()), ConnectionOptions: sdk.ConnectionOptions{ DialOptions: []grpc.DialOption{ @@ -188,75 +191,11 @@ func getTemporalClient(meta *temporalMetadata) (sdk.Client, error) { }) } -func parseTemporalMetadata(config *scalersconfig.ScalerConfig, logger logr.Logger) (*temporalMetadata, error) { - meta := &temporalMetadata{} - meta.activationLagThreshold = temporalDefaultActivationQueueLength - meta.targetQueueSize = temporalDefaultTargetQueueLength - - if config.TriggerMetadata["endpoint"] == "" { - return nil, errors.New("no Temporal gRPC endpoint provided") - } - meta.endpoint = config.TriggerMetadata["endpoint"] - - if config.TriggerMetadata["namespace"] == "" { - meta.namespace = temporalDefaultNamespace - } else { - meta.namespace = config.TriggerMetadata["namespace"] - } - - if size, ok := config.TriggerMetadata["targetQueueSize"]; ok { - queueSize, err := strconv.ParseInt(size, 10, 64) - if err != nil { - return nil, fmt.Errorf("invalid targetQueueSize - must be an integer") - } - meta.targetQueueSize = queueSize - } - - if size, ok := config.TriggerMetadata["activationTargetQueueSize"]; ok { - activationTargetQueueSize, err := strconv.ParseInt(size, 10, 64) - if err != nil { - return nil, fmt.Errorf("invalid activationTargetQueueSize - must be an integer") - } - meta.activationLagThreshold = activationTargetQueueSize - } - - if queueName, ok := config.TriggerMetadata["queueName"]; ok { - meta.queueName = queueName - } else { - return nil, errors.New("no queueName provided") - } - - // if buildIds is provided, it will be used to filter the queue and make sure - // selectAllActive and selectUnversioned are set to false to avoid considering - if buildIds, ok := config.TriggerMetadata["buildIds"]; ok && buildIds != "" { - meta.buildIDs = strings.Split(buildIds, ",") - } - - if val, ok := config.TriggerMetadata["selectAllActive"]; ok && val != "" { - allActive, err := strconv.ParseBool(val) - if err != nil { - meta.allActive = temporalDefaultSelectAllActive - logger.Error(err, "Error parsing Temoral queue metadata selectAllActive, using default %n", temporalDefaultSelectAllActive) - } else { - meta.allActive = allActive - } - } - - if val, ok := config.TriggerMetadata["selectUnversioned"]; ok && val != "" { - unversioned, err := strconv.ParseBool(val) - if err != nil { - meta.unversioned = temporalDefaultSelectUnversioned - logger.Error(err, "Error parsing Temoral queue metadata selectUnversioned, using default %n", temporalDefaultSelectUnversioned) - } else { - meta.unversioned = unversioned - } - } - - // optional, valide queueTypes are workflow, activity, nexus - if val, ok := config.TriggerMetadata["queueTypes"]; ok && val != "" { - meta.queueTypes = strings.Split(val, ",") +func parseTemporalMetadata(config *scalersconfig.ScalerConfig, _ logr.Logger) (*temporalMetadata, error) { + meta := &temporalMetadata{triggerIndex: config.TriggerIndex} + if err := config.TypedConfig(meta); err != nil { + return meta, fmt.Errorf("error parsing temporal metadata: %w", err) } - meta.triggerIndex = config.TriggerIndex return meta, nil } diff --git a/pkg/scalers/temporal_test.go b/pkg/scalers/temporal_test.go index e3c9ee70524..0b4cef2cbe3 100644 --- a/pkg/scalers/temporal_test.go +++ b/pkg/scalers/temporal_test.go @@ -48,22 +48,36 @@ var temporalMetricIdentifiers = []temporalMetricIdentifier{ {&testTemporalMetadata[4], 1, "s1-temporal-v2-default"}, } +func TestTemporalParseMetadata(t *testing.T) { + for _, testData := range testTemporalMetadata { + metadata := &scalersconfig.ScalerConfig{TriggerMetadata: testData.metadata} + _, err := parseTemporalMetadata(metadata, logger) + + if err != nil && !testData.isError { + t.Error("Expected success but got err", err) + } + if err == nil && testData.isError { + t.Error("Expected error but got success") + } + } +} + func TestTemporalGetMetricSpecForScaling(t *testing.T) { for _, testData := range temporalMetricIdentifiers { - ctx := context.Background() - meta, err := parseTemporalMetadata(&scalersconfig.ScalerConfig{ + metadata, err := parseTemporalMetadata(&scalersconfig.ScalerConfig{ TriggerMetadata: testData.metadataTestData.metadata, TriggerIndex: testData.triggerIndex, }, logger) + if err != nil { t.Fatal("Could not parse metadata:", err) } - mockTemporalScaler := temporalScaler{ - metadata: meta, + mockScaler := temporalScaler{ + metadata: metadata, } - - metricSpec := mockTemporalScaler.GetMetricSpecForScaling(ctx) + metricSpec := mockScaler.GetMetricSpecForScaling(context.Background()) metricName := metricSpec[0].External.Metric.Name + if metricName != testData.name { t.Error("Wrong External metric source name:", metricName) } @@ -72,24 +86,28 @@ func TestTemporalGetMetricSpecForScaling(t *testing.T) { func TestParseTemporalMetadata(t *testing.T) { cases := []struct { - name string - metadata map[string]string - wantMeta *temporalMetadata - wantErr bool + name string + metadata map[string]string + wantMeta *temporalMetadata + authParams map[string]string + wantErr bool }{ - { - name: "empty metadata", - wantMeta: nil, - wantErr: true, - }, { name: "empty queue name", metadata: map[string]string{ "endpoint": "test:7233", "namespace": "default", }, - wantMeta: nil, - wantErr: true, + wantMeta: &temporalMetadata{ + Endpoint: "test:7233", + Namespace: "default", + QueueName: "", + TargetQueueSize: 5, + ActivationLagThreshold: 0, + AllActive: true, + Unversioned: true, + }, + wantErr: true, }, { name: "empty namespace", @@ -98,10 +116,13 @@ func TestParseTemporalMetadata(t *testing.T) { "queueName": "testxx", }, wantMeta: &temporalMetadata{ - endpoint: "test:7233", - namespace: "default", - queueName: "testxx", - targetQueueSize: 5, + Endpoint: "test:7233", + Namespace: "default", + QueueName: "testxx", + TargetQueueSize: 5, + ActivationLagThreshold: 0, + AllActive: true, + Unversioned: true, }, wantErr: false, }, @@ -114,11 +135,56 @@ func TestParseTemporalMetadata(t *testing.T) { "activationTargetQueueSize": "12", }, wantMeta: &temporalMetadata{ - endpoint: "test:7233", - namespace: "default", - queueName: "testxx", - targetQueueSize: 5, - activationLagThreshold: 12, + Endpoint: "test:7233", + Namespace: "default", + QueueName: "testxx", + TargetQueueSize: 5, + ActivationLagThreshold: 12, + AllActive: true, + Unversioned: true, + }, + wantErr: false, + }, + { + name: "apiKey should not be empty", + metadata: map[string]string{ + "endpoint": "test:7233", + "namespace": "default", + "queueName": "testxx", + "apiKey": "test01", + }, + wantMeta: &temporalMetadata{ + Endpoint: "test:7233", + Namespace: "default", + QueueName: "testxx", + TargetQueueSize: 5, + ActivationLagThreshold: 0, + AllActive: true, + Unversioned: true, + ApiKey: "test01", + }, + authParams: map[string]string{ + "apiKey": "test01", + }, + wantErr: false, + }, + { + name: "queue type should not be empty", + metadata: map[string]string{ + "endpoint": "test:7233", + "namespace": "default", + "queueName": "testxx", + "queueTypes": "workflow,activity", + }, + wantMeta: &temporalMetadata{ + Endpoint: "test:7233", + Namespace: "default", + QueueName: "testxx", + TargetQueueSize: 5, + ActivationLagThreshold: 0, + AllActive: true, + Unversioned: true, + QueueTypes: []string{"workflow", "activity"}, }, wantErr: false, }, @@ -129,6 +195,7 @@ func TestParseTemporalMetadata(t *testing.T) { t.Run(c.name, func(t *testing.T) { config := &scalersconfig.ScalerConfig{ TriggerMetadata: c.metadata, + AuthParams: c.authParams, } meta, err := parseTemporalMetadata(config, logger) if c.wantErr == true && err != nil { @@ -140,3 +207,19 @@ func TestParseTemporalMetadata(t *testing.T) { }) } } + +func TestTemporalDefaultQueueTypes(t *testing.T) { + metadata, err := parseTemporalMetadata(&scalersconfig.ScalerConfig{ + TriggerMetadata: map[string]string{ + "endpoint": "localhost:7233", "queueName": "testcc", + }, + }, logger) + + assert.NoError(t, err, "error should be nil") + assert.Empty(t, metadata.QueueTypes, "queueTypes should be empty") + + assert.Len(t, getQueueTypes(metadata.QueueTypes), 3, "all queue types should be there") + + metadata.QueueTypes = []string{"workflow"} + assert.Len(t, getQueueTypes(metadata.QueueTypes), 1, "only one type should be there") +} From 984d1de5e1432b23f3c3fb48fa0808734660a5a1 Mon Sep 17 00:00:00 2001 From: Prajithp Date: Mon, 30 Sep 2024 22:35:22 +0530 Subject: [PATCH 04/18] support apiKey authentication Signed-off-by: Prajithp --- pkg/scalers/temporal.go | 63 ++++++++++++++++++++---------- pkg/scalers/temporal_test.go | 74 ++++++++++++++++++------------------ 2 files changed, 80 insertions(+), 57 deletions(-) diff --git a/pkg/scalers/temporal.go b/pkg/scalers/temporal.go index f1b5eb476c2..c6b42dbd1aa 100644 --- a/pkg/scalers/temporal.go +++ b/pkg/scalers/temporal.go @@ -12,6 +12,7 @@ import ( sdk "go.temporal.io/sdk/client" sdklog "go.temporal.io/sdk/log" "google.golang.org/grpc" + "google.golang.org/grpc/metadata" v2 "k8s.io/api/autoscaling/v2" "k8s.io/metrics/pkg/apis/external_metrics" ) @@ -32,16 +33,16 @@ type temporalScaler struct { } type temporalMetadata struct { - ActivationLagThreshold int64 `keda:"name=activationTargetQueueSize, order=triggerMetadata, default=0"` - Endpoint string `keda:"name=endpoint, order=triggerMetadata;resolvedEnv"` - Namespace string `keda:"name=namespace, order=triggerMetadata, default=default"` - TargetQueueSize int64 `keda:"name=targetQueueSize, order=triggerMetadata, default=5"` - QueueName string `keda:"name=queueName, order=triggerMetadata"` - QueueTypes []string `keda:"name=queueTypes, order=triggerMetadata, optional"` - BuildIDs []string `keda:"name=buildIds, order=triggerMetadata, optional"` - AllActive bool `keda:"name=selectAllActive, order=triggerMetadata, default=true"` - Unversioned bool `keda:"name=selectUnversioned, order=triggerMetadata, default=true"` - ApiKey string `keda:"name=apiKey, order=authParams;triggerMetadata, optional"` + ActivationTargetQueueSize int64 `keda:"name=activationTargetQueueSize, order=triggerMetadata, default=0"` + Endpoint string `keda:"name=endpoint, order=triggerMetadata;resolvedEnv"` + Namespace string `keda:"name=namespace, order=triggerMetadata, default=default"` + TargetQueueSize int64 `keda:"name=targetQueueSize, order=triggerMetadata, default=5"` + QueueName string `keda:"name=queueName, order=triggerMetadata"` + QueueTypes []string `keda:"name=queueTypes, order=triggerMetadata, optional"` + BuildIDs []string `keda:"name=buildIds, order=triggerMetadata, optional"` + AllActive bool `keda:"name=selectAllActive, order=triggerMetadata, default=true"` + Unversioned bool `keda:"name=selectUnversioned, order=triggerMetadata, default=true"` + APIKey string `keda:"name=apiKey, order=authParams;triggerMetadata, optional"` triggerIndex int } @@ -50,7 +51,7 @@ func (a *temporalMetadata) Validate() error { if a.TargetQueueSize <= 0 { return fmt.Errorf("targetQueueSize must be a positive number") } - if a.ActivationLagThreshold < 0 { + if a.ActivationTargetQueueSize < 0 { return fmt.Errorf("activationTargetQueueSize must be a positive number") } @@ -115,7 +116,7 @@ func (s *temporalScaler) GetMetricsAndActivity(ctx context.Context, metricName s metric := GenerateMetricInMili(metricName, float64(queueSize)) - return []external_metrics.ExternalMetricValue{metric}, queueSize > s.metadata.ActivationLagThreshold, nil + return []external_metrics.ExternalMetricValue{metric}, queueSize > s.metadata.ActivationTargetQueueSize, nil } func (s *temporalScaler) getQueueSize(ctx context.Context) (int64, error) { @@ -177,18 +178,40 @@ func getCombinedBacklogCount(description sdk.TaskQueueDescription) int64 { } func getTemporalClient(meta *temporalMetadata) (sdk.Client, error) { - return sdk.Dial(sdk.Options{ + options := sdk.Options{ HostPort: meta.Endpoint, Namespace: meta.Namespace, Logger: sdklog.NewStructuredLogger(slog.Default()), - ConnectionOptions: sdk.ConnectionOptions{ - DialOptions: []grpc.DialOption{ - grpc.WithConnectParams(grpc.ConnectParams{ - MinConnectTimeout: 5 * time.Second, - }), + } + + dialOptions := []grpc.DialOption{ + grpc.WithConnectParams(grpc.ConnectParams{ + MinConnectTimeout: 5 * time.Second, + }), + } + + if meta.APIKey != "" { + dialOptions = append(dialOptions, grpc.WithUnaryInterceptor( + func(ctx context.Context, method string, req any, reply any, + cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + return invoker( + metadata.AppendToOutgoingContext(ctx, "temporal-namespace", meta.Namespace), + method, + req, + reply, + cc, + opts..., + ) }, - }, - }) + )) + options.Credentials = sdk.NewAPIKeyStaticCredentials(meta.APIKey) + } + + options.ConnectionOptions = sdk.ConnectionOptions{ + DialOptions: dialOptions, + } + + return sdk.Dial(options) } func parseTemporalMetadata(config *scalersconfig.ScalerConfig, _ logr.Logger) (*temporalMetadata, error) { diff --git a/pkg/scalers/temporal_test.go b/pkg/scalers/temporal_test.go index 0b4cef2cbe3..e97c6c57cc6 100644 --- a/pkg/scalers/temporal_test.go +++ b/pkg/scalers/temporal_test.go @@ -99,13 +99,13 @@ func TestParseTemporalMetadata(t *testing.T) { "namespace": "default", }, wantMeta: &temporalMetadata{ - Endpoint: "test:7233", - Namespace: "default", - QueueName: "", - TargetQueueSize: 5, - ActivationLagThreshold: 0, - AllActive: true, - Unversioned: true, + Endpoint: "test:7233", + Namespace: "default", + QueueName: "", + TargetQueueSize: 5, + ActivationTargetQueueSize: 0, + AllActive: true, + Unversioned: true, }, wantErr: true, }, @@ -116,13 +116,13 @@ func TestParseTemporalMetadata(t *testing.T) { "queueName": "testxx", }, wantMeta: &temporalMetadata{ - Endpoint: "test:7233", - Namespace: "default", - QueueName: "testxx", - TargetQueueSize: 5, - ActivationLagThreshold: 0, - AllActive: true, - Unversioned: true, + Endpoint: "test:7233", + Namespace: "default", + QueueName: "testxx", + TargetQueueSize: 5, + ActivationTargetQueueSize: 0, + AllActive: true, + Unversioned: true, }, wantErr: false, }, @@ -135,13 +135,13 @@ func TestParseTemporalMetadata(t *testing.T) { "activationTargetQueueSize": "12", }, wantMeta: &temporalMetadata{ - Endpoint: "test:7233", - Namespace: "default", - QueueName: "testxx", - TargetQueueSize: 5, - ActivationLagThreshold: 12, - AllActive: true, - Unversioned: true, + Endpoint: "test:7233", + Namespace: "default", + QueueName: "testxx", + TargetQueueSize: 5, + ActivationTargetQueueSize: 12, + AllActive: true, + Unversioned: true, }, wantErr: false, }, @@ -154,14 +154,14 @@ func TestParseTemporalMetadata(t *testing.T) { "apiKey": "test01", }, wantMeta: &temporalMetadata{ - Endpoint: "test:7233", - Namespace: "default", - QueueName: "testxx", - TargetQueueSize: 5, - ActivationLagThreshold: 0, - AllActive: true, - Unversioned: true, - ApiKey: "test01", + Endpoint: "test:7233", + Namespace: "default", + QueueName: "testxx", + TargetQueueSize: 5, + ActivationTargetQueueSize: 0, + AllActive: true, + Unversioned: true, + APIKey: "test01", }, authParams: map[string]string{ "apiKey": "test01", @@ -177,14 +177,14 @@ func TestParseTemporalMetadata(t *testing.T) { "queueTypes": "workflow,activity", }, wantMeta: &temporalMetadata{ - Endpoint: "test:7233", - Namespace: "default", - QueueName: "testxx", - TargetQueueSize: 5, - ActivationLagThreshold: 0, - AllActive: true, - Unversioned: true, - QueueTypes: []string{"workflow", "activity"}, + Endpoint: "test:7233", + Namespace: "default", + QueueName: "testxx", + TargetQueueSize: 5, + ActivationTargetQueueSize: 0, + AllActive: true, + Unversioned: true, + QueueTypes: []string{"workflow", "activity"}, }, wantErr: false, }, From 58d899059628d2f3db81ab75c0ee24a3bd167199 Mon Sep 17 00:00:00 2001 From: Prajithp Date: Tue, 1 Oct 2024 09:37:21 +0530 Subject: [PATCH 05/18] use context Signed-off-by: Prajithp --- pkg/scalers/temporal.go | 8 ++++---- pkg/scaling/scalers_builder.go | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pkg/scalers/temporal.go b/pkg/scalers/temporal.go index c6b42dbd1aa..4709d97fb5c 100644 --- a/pkg/scalers/temporal.go +++ b/pkg/scalers/temporal.go @@ -58,7 +58,7 @@ func (a *temporalMetadata) Validate() error { return nil } -func NewTemporalScaler(config *scalersconfig.ScalerConfig) (Scaler, error) { +func NewTemporalScaler(ctx context.Context, config *scalersconfig.ScalerConfig) (Scaler, error) { logger := InitializeLogger(config, "temporal_scaler") metricType, err := GetMetricTargetType(config) @@ -71,7 +71,7 @@ func NewTemporalScaler(config *scalersconfig.ScalerConfig) (Scaler, error) { return nil, fmt.Errorf("failed to parse Temporal metadata: %w", err) } - c, err := getTemporalClient(meta) + c, err := getTemporalClient(ctx, meta) if err != nil { return nil, fmt.Errorf("failed to create Temporal client connection: %w", err) } @@ -177,7 +177,7 @@ func getCombinedBacklogCount(description sdk.TaskQueueDescription) int64 { return count } -func getTemporalClient(meta *temporalMetadata) (sdk.Client, error) { +func getTemporalClient(ctx context.Context, meta *temporalMetadata) (sdk.Client, error) { options := sdk.Options{ HostPort: meta.Endpoint, Namespace: meta.Namespace, @@ -211,7 +211,7 @@ func getTemporalClient(meta *temporalMetadata) (sdk.Client, error) { DialOptions: dialOptions, } - return sdk.Dial(options) + return sdk.DialContext(ctx, options) } func parseTemporalMetadata(config *scalersconfig.ScalerConfig, _ logr.Logger) (*temporalMetadata, error) { diff --git a/pkg/scaling/scalers_builder.go b/pkg/scaling/scalers_builder.go index f29b313b210..f85cf359840 100644 --- a/pkg/scaling/scalers_builder.go +++ b/pkg/scaling/scalers_builder.go @@ -252,7 +252,7 @@ func buildScaler(ctx context.Context, client client.Client, triggerType string, case "stan": return scalers.NewStanScaler(config) case "temporal": - return scalers.NewTemporalScaler(config) + return scalers.NewTemporalScaler(ctx, config) default: return nil, fmt.Errorf("no scaler found for type: %s", triggerType) } From 9aad76cee4ca1c6fd03809b70a60b87d535cb399 Mon Sep 17 00:00:00 2001 From: Prajithp Date: Fri, 4 Oct 2024 22:10:40 +0530 Subject: [PATCH 06/18] add MTLS auth option and some fixes Signed-off-by: Prajithp --- pkg/scalers/temporal.go | 43 ++++++++++++++++++------- pkg/scalers/temporal_test.go | 40 +++++++++++++++++++---- tests/scalers/temporal/temporal_test.go | 42 ++++++++---------------- 3 files changed, 78 insertions(+), 47 deletions(-) diff --git a/pkg/scalers/temporal.go b/pkg/scalers/temporal.go index 4709d97fb5c..ebe50158964 100644 --- a/pkg/scalers/temporal.go +++ b/pkg/scalers/temporal.go @@ -33,16 +33,22 @@ type temporalScaler struct { } type temporalMetadata struct { + Endpoint string `keda:"name=endpoint, order=triggerMetadata;resolvedEnv"` + Namespace string `keda:"name=namespace, order=triggerMetadata;resolvedEnv, default=default"` ActivationTargetQueueSize int64 `keda:"name=activationTargetQueueSize, order=triggerMetadata, default=0"` - Endpoint string `keda:"name=endpoint, order=triggerMetadata;resolvedEnv"` - Namespace string `keda:"name=namespace, order=triggerMetadata, default=default"` - TargetQueueSize int64 `keda:"name=targetQueueSize, order=triggerMetadata, default=5"` - QueueName string `keda:"name=queueName, order=triggerMetadata"` - QueueTypes []string `keda:"name=queueTypes, order=triggerMetadata, optional"` - BuildIDs []string `keda:"name=buildIds, order=triggerMetadata, optional"` - AllActive bool `keda:"name=selectAllActive, order=triggerMetadata, default=true"` - Unversioned bool `keda:"name=selectUnversioned, order=triggerMetadata, default=true"` - APIKey string `keda:"name=apiKey, order=authParams;triggerMetadata, optional"` + TargetQueueSize int64 `keda:"name=targetQueueSize, order=triggerMetadata, default=5"` + QueueName string `keda:"name=queueName, order=triggerMetadata;resolvedEnv"` + QueueTypes []string `keda:"name=queueTypes, order=triggerMetadata, optional"` + BuildIDs []string `keda:"name=buildIds, order=triggerMetadata;resolvedEnv, optional"` + AllActive bool `keda:"name=selectAllActive, order=triggerMetadata, default=true"` + Unversioned bool `keda:"name=selectUnversioned, order=triggerMetadata, default=true"` + APIKey string `keda:"name=apiKey, order=authParams;resolvedEnv;triggerMetadata, optional"` + + UnsafeSsl bool `keda:"name=unsafeSsl, order=triggerMetadata, optional"` + Cert string `keda:"name=cert, order=authParams;resolvedEnv, optional"` + Key string `keda:"name=key, order=authParams;resolvedEnv, optional"` + KeyPassword string `keda:"name=keyPassword, order=authParams;resolvedEnv, optional"` + CA string `keda:"name=ca, order=authParams;resolvedEnv, optional"` triggerIndex int } @@ -55,6 +61,10 @@ func (a *temporalMetadata) Validate() error { return fmt.Errorf("activationTargetQueueSize must be a positive number") } + if (a.Cert == "") != (a.Key == "") { + return fmt.Errorf("both cert and key must be provided when using TLS") + } + return nil } @@ -71,7 +81,7 @@ func NewTemporalScaler(ctx context.Context, config *scalersconfig.ScalerConfig) return nil, fmt.Errorf("failed to parse Temporal metadata: %w", err) } - c, err := getTemporalClient(ctx, meta) + c, err := getTemporalClient(ctx, meta, logger) if err != nil { return nil, fmt.Errorf("failed to create Temporal client connection: %w", err) } @@ -177,11 +187,12 @@ func getCombinedBacklogCount(description sdk.TaskQueueDescription) int64 { return count } -func getTemporalClient(ctx context.Context, meta *temporalMetadata) (sdk.Client, error) { +func getTemporalClient(ctx context.Context, meta *temporalMetadata, log logr.Logger) (sdk.Client, error) { + logHandler := logr.ToSlogHandler(log) options := sdk.Options{ HostPort: meta.Endpoint, Namespace: meta.Namespace, - Logger: sdklog.NewStructuredLogger(slog.Default()), + Logger: sdklog.NewStructuredLogger(slog.New(logHandler)), } dialOptions := []grpc.DialOption{ @@ -211,6 +222,14 @@ func getTemporalClient(ctx context.Context, meta *temporalMetadata) (sdk.Client, DialOptions: dialOptions, } + if meta.Cert != "" && meta.Key != "" { + tlsConfig, err := kedautil.NewTLSConfigWithPassword(meta.Cert, meta.Key, meta.KeyPassword, meta.CA, meta.UnsafeSsl) + if err != nil { + return nil, err + } + options.ConnectionOptions.TLS = tlsConfig + } + return sdk.DialContext(ctx, options) } diff --git a/pkg/scalers/temporal_test.go b/pkg/scalers/temporal_test.go index e97c6c57cc6..212934aebfb 100644 --- a/pkg/scalers/temporal_test.go +++ b/pkg/scalers/temporal_test.go @@ -86,11 +86,12 @@ func TestTemporalGetMetricSpecForScaling(t *testing.T) { func TestParseTemporalMetadata(t *testing.T) { cases := []struct { - name string - metadata map[string]string - wantMeta *temporalMetadata - authParams map[string]string - wantErr bool + name string + metadata map[string]string + wantMeta *temporalMetadata + authParams map[string]string + resolvedEnv map[string]string + wantErr bool }{ { name: "empty queue name", @@ -151,7 +152,6 @@ func TestParseTemporalMetadata(t *testing.T) { "endpoint": "test:7233", "namespace": "default", "queueName": "testxx", - "apiKey": "test01", }, wantMeta: &temporalMetadata{ Endpoint: "test:7233", @@ -188,6 +188,33 @@ func TestParseTemporalMetadata(t *testing.T) { }, wantErr: false, }, + { + name: "read config from env", + resolvedEnv: map[string]string{ + "endpoint": "test:7233", + "namespace": "default", + "queueName": "testxx", + }, + metadata: map[string]string{ + "endpointFromEnv": "endpoint", + "namespaceFromEnv": "namespace", + "queueNameFromEnv": "queueName", + }, + wantMeta: &temporalMetadata{ + Endpoint: "test:7233", + Namespace: "default", + QueueName: "testxx", + TargetQueueSize: 5, + ActivationTargetQueueSize: 0, + AllActive: true, + Unversioned: true, + APIKey: "test01", + }, + authParams: map[string]string{ + "apiKey": "test01", + }, + wantErr: false, + }, } for _, testCase := range cases { @@ -196,6 +223,7 @@ func TestParseTemporalMetadata(t *testing.T) { config := &scalersconfig.ScalerConfig{ TriggerMetadata: c.metadata, AuthParams: c.authParams, + ResolvedEnv: c.resolvedEnv, } meta, err := parseTemporalMetadata(config, logger) if c.wantErr == true && err != nil { diff --git a/tests/scalers/temporal/temporal_test.go b/tests/scalers/temporal/temporal_test.go index 54c281d8aa1..d81c965720f 100644 --- a/tests/scalers/temporal/temporal_test.go +++ b/tests/scalers/temporal/temporal_test.go @@ -71,40 +71,22 @@ spec: app: temporal spec: containers: - - name: ubuntu - image: ubuntu:latest + - name: temporal + image: temporalio/admin-tools:latest command: ["bash", "-c"] args: - | - apt-get update && apt-get install -y curl && \ - curl -sSf https://temporal.download/cli.sh | sh && \ - PATH="\$PATH:/root/.temporalio/bin" >> ~/.bashrc && \ - source ~/.bashrc && \ temporal server start-dev --namespace v2 --ip 0.0.0.0 ports: - containerPort: 7233 livenessProbe: - exec: - command: - - /bin/sh - - -ec - - test $(ps -ef | grep -v grep | grep "temporal server" | wc -l) -eq 1 + tcpSocket: + port: 7233 failureThreshold: 5 initialDelaySeconds: 10 periodSeconds: 30 successThreshold: 1 - timeoutSeconds: 1 - readinessProbe: - exec: - command: - - /bin/sh - - -ec - - test $(ps -ef | grep -v grep | grep "temporal server" | wc -l) -eq 1 - failureThreshold: 5 - initialDelaySeconds: 10 - periodSeconds: 30 - successThreshold: 1 - timeoutSeconds: 1 + timeoutSeconds: 2 ` scaledObjectTemplate = ` @@ -184,17 +166,17 @@ spec: backoffLimit: 4 ` - jobWorkeFlowTemplate = ` + jobWorkFlowTemplate = ` apiVersion: batch/v1 kind: Job metadata: - name: workerflow + name: workflow namespace: {{.TestNamespace}} spec: template: spec: containers: - - name: workerflow + - name: workflow image: "prajithp/temporal-sample:1.0.0" imagePullPolicy: Always env: @@ -207,6 +189,7 @@ spec: func getTemplateData() (templateData, []Template) { return templateData{ + TestNamespace: testNamespace, TemporalDeploymentName: TemporalDeploymentName, ScaledObjectName: scaledObjectName, @@ -236,17 +219,17 @@ func TestTemporalScaler(t *testing.T) { func testActivation(t *testing.T, kc *kubernetes.Clientset, data templateData) { t.Log("--- testing activation ---") - KubectlApplyWithTemplate(t, data, "jobWorkFlowActivation", jobWorkeFlowTemplate) + KubectlApplyWithTemplate(t, data, "jobWorkFlowActivation", jobWorkFlowTemplate) assert.True(t, WaitForJobCount(t, kc, testNamespace, 1, 60, 3), "job count in namespace should be 1") AssertReplicaCountNotChangeDuringTimePeriod(t, kc, deploymentName, testNamespace, 0, 180) - KubectlDeleteWithTemplate(t, data, "jobWorkFlowActivation", jobWorkeFlowTemplate) + KubectlDeleteWithTemplate(t, data, "jobWorkFlowActivation", jobWorkFlowTemplate) } func testScaleOut(t *testing.T, kc *kubernetes.Clientset, data templateData) { t.Log("--- testing scale out ---") - KubectlApplyWithTemplate(t, data, "jobWorkFlow", jobWorkeFlowTemplate) + KubectlApplyWithTemplate(t, data, "jobWorkFlow", jobWorkFlowTemplate) assert.True(t, WaitForJobCount(t, kc, testNamespace, 1, 60, 3), "job count in namespace should be 1") assert.True(t, WaitForDeploymentReplicaReadyCount(t, kc, deploymentName, testNamespace, 1, 60, 3), @@ -262,4 +245,5 @@ func testScaleIn(t *testing.T, kc *kubernetes.Clientset, data templateData) { assert.True(t, WaitForDeploymentReplicaReadyCount(t, kc, deploymentName, testNamespace, 0, 60, 5), "replica count should be %d after 5 minutes", 0) + } From 5159eb6f9027e583c6165fe66c5d7485aea8b17f Mon Sep 17 00:00:00 2001 From: Prajithp Date: Mon, 7 Oct 2024 17:49:00 +0530 Subject: [PATCH 07/18] update e2e test to use official image Signed-off-by: Prajithp --- tests/scalers/temporal/temporal_test.go | 68 +++++++++++-------------- 1 file changed, 30 insertions(+), 38 deletions(-) diff --git a/tests/scalers/temporal/temporal_test.go b/tests/scalers/temporal/temporal_test.go index d81c965720f..b6df6b433b7 100644 --- a/tests/scalers/temporal/temporal_test.go +++ b/tests/scalers/temporal/temporal_test.go @@ -29,6 +29,8 @@ var ( ) type templateData struct { + WorkFlowCommand string + WorkFlowIterations int DeploymentName string TestNamespace string TemporalDeploymentName string @@ -111,7 +113,7 @@ spec: - type: temporal metadata: namespace: default - queueName: hello-task-queue + queueName: "workflow_with_single_noop_activity:test" targetQueueSize: "2" activationTargetQueueSize: "3" endpoint: {{.TemporalDeploymentName}}.{{.TestNamespace}}.svc.cluster.local:7233 @@ -134,36 +136,19 @@ spec: metadata: labels: app: {{.DeploymentName}} - spec: - containers: - - name: nginx - image: nginxinc/nginx-unprivileged - ports: - - containerPort: 80 -` - - jobWorkerTemplate = ` -apiVersion: batch/v1 -kind: Job -metadata: - name: worker - namespace: {{.TestNamespace}} -spec: - template: spec: containers: - name: worker - image: "prajithp/temporal-sample:1.0.0" + image: "temporaliotest/omes:go-ci-latest" imagePullPolicy: Always - env: - - name: TEMPORAL_ADDR - value: {{.TemporalDeploymentName}}.{{.TestNamespace}}.svc.cluster.local:7233 - - name: TEMPORAL_NAMESPACE - value: default - - name: MODE - value: WORKER - restartPolicy: OnFailure - backoffLimit: 4 + command: ["/app/temporal-omes"] + args: + - "run-worker" + - "--language=go" + - "--server-address={{.TemporalDeploymentName}}.{{.TestNamespace}}.svc.cluster.local:7233" + - "--run-id=test" + - "--scenario=workflow_with_single_noop_activity" + - "--dir-name=prepared" ` jobWorkFlowTemplate = ` @@ -177,19 +162,26 @@ spec: spec: containers: - name: workflow - image: "prajithp/temporal-sample:1.0.0" + image: "temporaliotest/omes:go-ci-latest" imagePullPolicy: Always - env: - - name: TEMPORAL_ADDR - value: {{.TemporalDeploymentName}}.{{.TestNamespace}}.svc.cluster.local:7233 + command: ["/app/temporal-omes"] + args: + - "{{.WorkFlowCommand}}" + {{- if eq .WorkFlowCommand "run-scenario"}} + - "--iterations={{.WorkFlowIterations}}" + {{- end}} + - "--scenario=workflow_with_single_noop_activity" + - "--run-id=test" + - "--server-address={{.TemporalDeploymentName}}.{{.TestNamespace}}.svc.cluster.local:7233" restartPolicy: OnFailure - backoffLimit: 4 + backoffLimit: 10 ` ) func getTemplateData() (templateData, []Template) { return templateData{ - + WorkFlowCommand: "run-scenario", + WorkFlowIterations: 2, TestNamespace: testNamespace, TemporalDeploymentName: TemporalDeploymentName, ScaledObjectName: scaledObjectName, @@ -224,11 +216,16 @@ func testActivation(t *testing.T, kc *kubernetes.Clientset, data templateData) { AssertReplicaCountNotChangeDuringTimePeriod(t, kc, deploymentName, testNamespace, 0, 180) KubectlDeleteWithTemplate(t, data, "jobWorkFlowActivation", jobWorkFlowTemplate) + data.WorkFlowCommand = "cleanup-scenario" + KubectlApplyWithTemplate(t, data, "jobWorkflowCleanup", jobWorkFlowTemplate) + assert.True(t, WaitForJobCount(t, kc, testNamespace, 1, 60, 3), "job count in namespace should be 1") + KubectlDeleteWithTemplate(t, data, "jobWorkflowCleanup", jobWorkFlowTemplate) } func testScaleOut(t *testing.T, kc *kubernetes.Clientset, data templateData) { t.Log("--- testing scale out ---") + data.WorkFlowIterations = 3 KubectlApplyWithTemplate(t, data, "jobWorkFlow", jobWorkFlowTemplate) assert.True(t, WaitForJobCount(t, kc, testNamespace, 1, 60, 3), "job count in namespace should be 1") @@ -239,11 +236,6 @@ func testScaleOut(t *testing.T, kc *kubernetes.Clientset, data templateData) { func testScaleIn(t *testing.T, kc *kubernetes.Clientset, data templateData) { t.Log("--- testing scale in ---") - KubectlApplyWithTemplate(t, data, "jobWorker", jobWorkerTemplate) - // workflow is already waiting for response from worker - assert.True(t, WaitForJobCount(t, kc, testNamespace, 2, 60, 3), "job count in namespace should be 2") - assert.True(t, WaitForDeploymentReplicaReadyCount(t, kc, deploymentName, testNamespace, 0, 60, 5), "replica count should be %d after 5 minutes", 0) - } From 5c8b3e6f3ecc987e857430beae6e4c7ce638406f Mon Sep 17 00:00:00 2001 From: Prajithp Date: Sun, 13 Oct 2024 09:23:25 +0530 Subject: [PATCH 08/18] rename metadata variables Signed-off-by: Prajithp --- pkg/scalers/temporal.go | 12 ++++----- pkg/scalers/temporal_test.go | 36 ++++++++++++------------- tests/scalers/temporal/temporal_test.go | 2 +- 3 files changed, 25 insertions(+), 25 deletions(-) diff --git a/pkg/scalers/temporal.go b/pkg/scalers/temporal.go index ebe50158964..fd1de0e2622 100644 --- a/pkg/scalers/temporal.go +++ b/pkg/scalers/temporal.go @@ -37,9 +37,9 @@ type temporalMetadata struct { Namespace string `keda:"name=namespace, order=triggerMetadata;resolvedEnv, default=default"` ActivationTargetQueueSize int64 `keda:"name=activationTargetQueueSize, order=triggerMetadata, default=0"` TargetQueueSize int64 `keda:"name=targetQueueSize, order=triggerMetadata, default=5"` - QueueName string `keda:"name=queueName, order=triggerMetadata;resolvedEnv"` + TaskQueue string `keda:"name=taskQueue, order=triggerMetadata;resolvedEnv"` QueueTypes []string `keda:"name=queueTypes, order=triggerMetadata, optional"` - BuildIDs []string `keda:"name=buildIds, order=triggerMetadata;resolvedEnv, optional"` + BuildID string `keda:"name=buildId, order=triggerMetadata;resolvedEnv, optional"` AllActive bool `keda:"name=selectAllActive, order=triggerMetadata, default=true"` Unversioned bool `keda:"name=selectUnversioned, order=triggerMetadata, default=true"` APIKey string `keda:"name=apiKey, order=authParams;resolvedEnv;triggerMetadata, optional"` @@ -102,7 +102,7 @@ func (s *temporalScaler) Close(_ context.Context) error { } func (s *temporalScaler) GetMetricSpecForScaling(context.Context) []v2.MetricSpec { - metricName := kedautil.NormalizeString(fmt.Sprintf("temporal-%s-%s", s.metadata.Namespace, s.metadata.QueueName)) + metricName := kedautil.NormalizeString(fmt.Sprintf("temporal-%s-%s", s.metadata.Namespace, s.metadata.TaskQueue)) externalMetric := &v2.ExternalMetricSource{ Metric: v2.MetricIdentifier{ Name: GenerateMetricNameWithIndex(s.metadata.triggerIndex, metricName), @@ -131,18 +131,18 @@ func (s *temporalScaler) GetMetricsAndActivity(ctx context.Context, metricName s func (s *temporalScaler) getQueueSize(ctx context.Context) (int64, error) { var selection *sdk.TaskQueueVersionSelection - if s.metadata.AllActive || s.metadata.Unversioned || len(s.metadata.BuildIDs) > 0 { + if s.metadata.AllActive || s.metadata.Unversioned || s.metadata.BuildID != "" { selection = &sdk.TaskQueueVersionSelection{ AllActive: s.metadata.AllActive, Unversioned: s.metadata.Unversioned, - BuildIDs: s.metadata.BuildIDs, + BuildIDs: []string{s.metadata.BuildID}, } } queueType := getQueueTypes(s.metadata.QueueTypes) resp, err := s.tcl.DescribeTaskQueueEnhanced(ctx, sdk.DescribeTaskQueueEnhancedOptions{ - TaskQueue: s.metadata.QueueName, + TaskQueue: s.metadata.TaskQueue, ReportStats: true, Versions: selection, TaskQueueTypes: queueType, diff --git a/pkg/scalers/temporal_test.go b/pkg/scalers/temporal_test.go index 212934aebfb..8f352c7ae75 100644 --- a/pkg/scalers/temporal_test.go +++ b/pkg/scalers/temporal_test.go @@ -31,16 +31,16 @@ type temporalMetricIdentifier struct { var testTemporalMetadata = []parseTemporalMetadataTestData{ // nothing passed {map[string]string{}, true}, - // Missing queueName, should fail + // Missing taskQueue, should fail {map[string]string{"endpoint": temporalEndpoint, "namespace": temporalNamespace}, true}, // Missing namespace, should success - {map[string]string{"endpoint": temporalEndpoint, "queueName": temporalQueueName}, false}, + {map[string]string{"endpoint": temporalEndpoint, "taskQueue": temporalQueueName}, false}, // Missing endpoint, should fail - {map[string]string{"queueName": temporalQueueName, "namespace": temporalNamespace}, true}, + {map[string]string{"taskQueue": temporalQueueName, "namespace": temporalNamespace}, true}, // All good. - {map[string]string{"endpoint": temporalEndpoint, "queueName": temporalQueueName, "namespace": temporalNamespace}, false}, + {map[string]string{"endpoint": temporalEndpoint, "taskQueue": temporalQueueName, "namespace": temporalNamespace}, false}, // All good + activationLagThreshold - {map[string]string{"endpoint": temporalEndpoint, "queueName": temporalQueueName, "namespace": temporalNamespace, "activationTargetQueueSize": "10"}, false}, + {map[string]string{"endpoint": temporalEndpoint, "taskQueue": temporalQueueName, "namespace": temporalNamespace, "activationTargetQueueSize": "10"}, false}, } var temporalMetricIdentifiers = []temporalMetricIdentifier{ @@ -102,7 +102,7 @@ func TestParseTemporalMetadata(t *testing.T) { wantMeta: &temporalMetadata{ Endpoint: "test:7233", Namespace: "default", - QueueName: "", + TaskQueue: "", TargetQueueSize: 5, ActivationTargetQueueSize: 0, AllActive: true, @@ -114,12 +114,12 @@ func TestParseTemporalMetadata(t *testing.T) { name: "empty namespace", metadata: map[string]string{ "endpoint": "test:7233", - "queueName": "testxx", + "taskQueue": "testxx", }, wantMeta: &temporalMetadata{ Endpoint: "test:7233", Namespace: "default", - QueueName: "testxx", + TaskQueue: "testxx", TargetQueueSize: 5, ActivationTargetQueueSize: 0, AllActive: true, @@ -132,13 +132,13 @@ func TestParseTemporalMetadata(t *testing.T) { metadata: map[string]string{ "endpoint": "test:7233", "namespace": "default", - "queueName": "testxx", + "taskQueue": "testxx", "activationTargetQueueSize": "12", }, wantMeta: &temporalMetadata{ Endpoint: "test:7233", Namespace: "default", - QueueName: "testxx", + TaskQueue: "testxx", TargetQueueSize: 5, ActivationTargetQueueSize: 12, AllActive: true, @@ -151,12 +151,12 @@ func TestParseTemporalMetadata(t *testing.T) { metadata: map[string]string{ "endpoint": "test:7233", "namespace": "default", - "queueName": "testxx", + "taskQueue": "testxx", }, wantMeta: &temporalMetadata{ Endpoint: "test:7233", Namespace: "default", - QueueName: "testxx", + TaskQueue: "testxx", TargetQueueSize: 5, ActivationTargetQueueSize: 0, AllActive: true, @@ -173,13 +173,13 @@ func TestParseTemporalMetadata(t *testing.T) { metadata: map[string]string{ "endpoint": "test:7233", "namespace": "default", - "queueName": "testxx", + "taskQueue": "testxx", "queueTypes": "workflow,activity", }, wantMeta: &temporalMetadata{ Endpoint: "test:7233", Namespace: "default", - QueueName: "testxx", + TaskQueue: "testxx", TargetQueueSize: 5, ActivationTargetQueueSize: 0, AllActive: true, @@ -193,17 +193,17 @@ func TestParseTemporalMetadata(t *testing.T) { resolvedEnv: map[string]string{ "endpoint": "test:7233", "namespace": "default", - "queueName": "testxx", + "taskQueue": "testxx", }, metadata: map[string]string{ "endpointFromEnv": "endpoint", "namespaceFromEnv": "namespace", - "queueNameFromEnv": "queueName", + "taskQueueFromEnv": "taskQueue", }, wantMeta: &temporalMetadata{ Endpoint: "test:7233", Namespace: "default", - QueueName: "testxx", + TaskQueue: "testxx", TargetQueueSize: 5, ActivationTargetQueueSize: 0, AllActive: true, @@ -239,7 +239,7 @@ func TestParseTemporalMetadata(t *testing.T) { func TestTemporalDefaultQueueTypes(t *testing.T) { metadata, err := parseTemporalMetadata(&scalersconfig.ScalerConfig{ TriggerMetadata: map[string]string{ - "endpoint": "localhost:7233", "queueName": "testcc", + "endpoint": "localhost:7233", "taskQueue": "testcc", }, }, logger) diff --git a/tests/scalers/temporal/temporal_test.go b/tests/scalers/temporal/temporal_test.go index b6df6b433b7..4a178bbc435 100644 --- a/tests/scalers/temporal/temporal_test.go +++ b/tests/scalers/temporal/temporal_test.go @@ -113,7 +113,7 @@ spec: - type: temporal metadata: namespace: default - queueName: "workflow_with_single_noop_activity:test" + taskQueue: "workflow_with_single_noop_activity:test" targetQueueSize: "2" activationTargetQueueSize: "3" endpoint: {{.TemporalDeploymentName}}.{{.TestNamespace}}.svc.cluster.local:7233 From e946a54e03a4500245dc008c4fd6da9b9b295a07 Mon Sep 17 00:00:00 2001 From: Prajithp Date: Sun, 13 Oct 2024 09:31:26 +0530 Subject: [PATCH 09/18] fix temporal server override command Signed-off-by: Prajithp --- tests/scalers/temporal/temporal_test.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/scalers/temporal/temporal_test.go b/tests/scalers/temporal/temporal_test.go index 4a178bbc435..b092fd201a3 100644 --- a/tests/scalers/temporal/temporal_test.go +++ b/tests/scalers/temporal/temporal_test.go @@ -75,10 +75,7 @@ spec: containers: - name: temporal image: temporalio/admin-tools:latest - command: ["bash", "-c"] - args: - - | - temporal server start-dev --namespace v2 --ip 0.0.0.0 + command: ["temporal", "server", "start-dev", "--namespace", "v2", "--ip", "0.0.0.0"] ports: - containerPort: 7233 livenessProbe: From 7390335318618c8600a5f6b43653898346713d3d Mon Sep 17 00:00:00 2001 From: Prajithp Date: Mon, 14 Oct 2024 07:50:02 +0530 Subject: [PATCH 10/18] remove namespace from cli args Signed-off-by: Prajithp --- tests/scalers/temporal/temporal_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/scalers/temporal/temporal_test.go b/tests/scalers/temporal/temporal_test.go index b092fd201a3..27a6ed8b9c1 100644 --- a/tests/scalers/temporal/temporal_test.go +++ b/tests/scalers/temporal/temporal_test.go @@ -75,7 +75,7 @@ spec: containers: - name: temporal image: temporalio/admin-tools:latest - command: ["temporal", "server", "start-dev", "--namespace", "v2", "--ip", "0.0.0.0"] + command: ["temporal", "server", "start-dev", "--ip", "0.0.0.0"] ports: - containerPort: 7233 livenessProbe: From 66a373bf2006774e596a3449c01702fd569a9635 Mon Sep 17 00:00:00 2001 From: Prajith Date: Thu, 17 Oct 2024 08:24:35 +0530 Subject: [PATCH 11/18] Update tests/scalers/temporal/temporal_test.go Co-authored-by: Jorge Turrado Ferrero Signed-off-by: Prajith --- tests/scalers/temporal/temporal_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/scalers/temporal/temporal_test.go b/tests/scalers/temporal/temporal_test.go index 27a6ed8b9c1..5f234d09827 100644 --- a/tests/scalers/temporal/temporal_test.go +++ b/tests/scalers/temporal/temporal_test.go @@ -211,7 +211,7 @@ func testActivation(t *testing.T, kc *kubernetes.Clientset, data templateData) { KubectlApplyWithTemplate(t, data, "jobWorkFlowActivation", jobWorkFlowTemplate) assert.True(t, WaitForJobCount(t, kc, testNamespace, 1, 60, 3), "job count in namespace should be 1") - AssertReplicaCountNotChangeDuringTimePeriod(t, kc, deploymentName, testNamespace, 0, 180) + AssertReplicaCountNotChangeDuringTimePeriod(t, kc, deploymentName, testNamespace, 0, 60) KubectlDeleteWithTemplate(t, data, "jobWorkFlowActivation", jobWorkFlowTemplate) data.WorkFlowCommand = "cleanup-scenario" KubectlApplyWithTemplate(t, data, "jobWorkflowCleanup", jobWorkFlowTemplate) From 7681beb334cb5265b4ed4f379c2147c89293bc98 Mon Sep 17 00:00:00 2001 From: Prajith Date: Thu, 17 Oct 2024 08:29:47 +0530 Subject: [PATCH 12/18] Update tests/scalers/temporal/temporal_test.go Co-authored-by: Jorge Turrado Ferrero Signed-off-by: Prajith --- tests/scalers/temporal/temporal_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/scalers/temporal/temporal_test.go b/tests/scalers/temporal/temporal_test.go index 5f234d09827..fd0afc14e2a 100644 --- a/tests/scalers/temporal/temporal_test.go +++ b/tests/scalers/temporal/temporal_test.go @@ -22,7 +22,7 @@ const ( var ( testNamespace = fmt.Sprintf("%s-ns", testName) - TemporalDeploymentName = fmt.Sprintf("temporal-%s", testName) + temporalDeploymentName = fmt.Sprintf("temporal-%s", testName) scaledObjectName = fmt.Sprintf("%s-so", testName) deploymentName = fmt.Sprintf("%s", testName) From d1aa803d45b246622f30cb27f704c6991c1ee933 Mon Sep 17 00:00:00 2001 From: Prajith Date: Thu, 17 Oct 2024 08:30:02 +0530 Subject: [PATCH 13/18] Update tests/scalers/temporal/temporal_test.go Co-authored-by: Jorge Turrado Ferrero Signed-off-by: Prajith --- tests/scalers/temporal/temporal_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/scalers/temporal/temporal_test.go b/tests/scalers/temporal/temporal_test.go index fd0afc14e2a..d54499a03ef 100644 --- a/tests/scalers/temporal/temporal_test.go +++ b/tests/scalers/temporal/temporal_test.go @@ -180,7 +180,7 @@ func getTemplateData() (templateData, []Template) { WorkFlowCommand: "run-scenario", WorkFlowIterations: 2, TestNamespace: testNamespace, - TemporalDeploymentName: TemporalDeploymentName, + TemporalDeploymentName: temporalDeploymentName, ScaledObjectName: scaledObjectName, DeploymentName: deploymentName, }, []Template{ From 4cdecfb59642110093ff316bcfea712ad8662779 Mon Sep 17 00:00:00 2001 From: Prajithp Date: Thu, 17 Oct 2024 12:11:07 +0530 Subject: [PATCH 14/18] add MinConnectTimeout option Signed-off-by: Prajithp --- pkg/scalers/temporal.go | 15 ++++++++++----- pkg/scalers/temporal_test.go | 12 ++++++++++-- tests/scalers/temporal/temporal_test.go | 2 +- 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/pkg/scalers/temporal.go b/pkg/scalers/temporal.go index fd1de0e2622..c5ea4608954 100644 --- a/pkg/scalers/temporal.go +++ b/pkg/scalers/temporal.go @@ -43,12 +43,13 @@ type temporalMetadata struct { AllActive bool `keda:"name=selectAllActive, order=triggerMetadata, default=true"` Unversioned bool `keda:"name=selectUnversioned, order=triggerMetadata, default=true"` APIKey string `keda:"name=apiKey, order=authParams;resolvedEnv;triggerMetadata, optional"` + MinConnectTimeout int `keda:"name=minConnectTimeout, order=triggerMetadata, default=5"` UnsafeSsl bool `keda:"name=unsafeSsl, order=triggerMetadata, optional"` - Cert string `keda:"name=cert, order=authParams;resolvedEnv, optional"` - Key string `keda:"name=key, order=authParams;resolvedEnv, optional"` - KeyPassword string `keda:"name=keyPassword, order=authParams;resolvedEnv, optional"` - CA string `keda:"name=ca, order=authParams;resolvedEnv, optional"` + Cert string `keda:"name=cert, order=authParams, optional"` + Key string `keda:"name=key, order=authParams, optional"` + KeyPassword string `keda:"name=keyPassword, order=authParams, optional"` + CA string `keda:"name=ca, order=authParams, optional"` triggerIndex int } @@ -65,6 +66,10 @@ func (a *temporalMetadata) Validate() error { return fmt.Errorf("both cert and key must be provided when using TLS") } + if a.MinConnectTimeout < 0 { + return fmt.Errorf("minConnectTimeout must be a positive number") + } + return nil } @@ -197,7 +202,7 @@ func getTemporalClient(ctx context.Context, meta *temporalMetadata, log logr.Log dialOptions := []grpc.DialOption{ grpc.WithConnectParams(grpc.ConnectParams{ - MinConnectTimeout: 5 * time.Second, + MinConnectTimeout: time.Duration(meta.MinConnectTimeout) * time.Second, }), } diff --git a/pkg/scalers/temporal_test.go b/pkg/scalers/temporal_test.go index 8f352c7ae75..a9cee20d1a6 100644 --- a/pkg/scalers/temporal_test.go +++ b/pkg/scalers/temporal_test.go @@ -37,6 +37,8 @@ var testTemporalMetadata = []parseTemporalMetadataTestData{ {map[string]string{"endpoint": temporalEndpoint, "taskQueue": temporalQueueName}, false}, // Missing endpoint, should fail {map[string]string{"taskQueue": temporalQueueName, "namespace": temporalNamespace}, true}, + // invalid minConnectTimeout + {map[string]string{"endpoint": temporalEndpoint, "taskQueue": temporalQueueName, "namespace": temporalNamespace, "minConnectTimeout": "-1"}, true}, // All good. {map[string]string{"endpoint": temporalEndpoint, "taskQueue": temporalQueueName, "namespace": temporalNamespace}, false}, // All good + activationLagThreshold @@ -44,8 +46,8 @@ var testTemporalMetadata = []parseTemporalMetadataTestData{ } var temporalMetricIdentifiers = []temporalMetricIdentifier{ - {&testTemporalMetadata[4], 0, "s0-temporal-v2-default"}, - {&testTemporalMetadata[4], 1, "s1-temporal-v2-default"}, + {&testTemporalMetadata[5], 0, "s0-temporal-v2-default"}, + {&testTemporalMetadata[5], 1, "s1-temporal-v2-default"}, } func TestTemporalParseMetadata(t *testing.T) { @@ -107,6 +109,7 @@ func TestParseTemporalMetadata(t *testing.T) { ActivationTargetQueueSize: 0, AllActive: true, Unversioned: true, + MinConnectTimeout: 5, }, wantErr: true, }, @@ -124,6 +127,7 @@ func TestParseTemporalMetadata(t *testing.T) { ActivationTargetQueueSize: 0, AllActive: true, Unversioned: true, + MinConnectTimeout: 5, }, wantErr: false, }, @@ -143,6 +147,7 @@ func TestParseTemporalMetadata(t *testing.T) { ActivationTargetQueueSize: 12, AllActive: true, Unversioned: true, + MinConnectTimeout: 5, }, wantErr: false, }, @@ -162,6 +167,7 @@ func TestParseTemporalMetadata(t *testing.T) { AllActive: true, Unversioned: true, APIKey: "test01", + MinConnectTimeout: 5, }, authParams: map[string]string{ "apiKey": "test01", @@ -185,6 +191,7 @@ func TestParseTemporalMetadata(t *testing.T) { AllActive: true, Unversioned: true, QueueTypes: []string{"workflow", "activity"}, + MinConnectTimeout: 5, }, wantErr: false, }, @@ -209,6 +216,7 @@ func TestParseTemporalMetadata(t *testing.T) { AllActive: true, Unversioned: true, APIKey: "test01", + MinConnectTimeout: 5, }, authParams: map[string]string{ "apiKey": "test01", diff --git a/tests/scalers/temporal/temporal_test.go b/tests/scalers/temporal/temporal_test.go index d54499a03ef..032fa7b6f6c 100644 --- a/tests/scalers/temporal/temporal_test.go +++ b/tests/scalers/temporal/temporal_test.go @@ -196,7 +196,7 @@ func TestTemporalScaler(t *testing.T) { KubectlApplyWithTemplate(t, data, "temporalServiceTemplate", temporalServiceTemplate) KubectlApplyWithTemplate(t, data, "temporalDeploymentTemplate", temporalDeploymentTemplate) - assert.True(t, WaitForDeploymentReplicaReadyCount(t, kc, TemporalDeploymentName, testNamespace, 1, 30, 4), "temporal is not in a ready state") + assert.True(t, WaitForDeploymentReplicaReadyCount(t, kc, temporalDeploymentName, testNamespace, 1, 30, 4), "temporal is not in a ready state") KubectlApplyMultipleWithTemplate(t, data, templates) testActivation(t, kc, data) From ba2049ab3ad5b63c9a1c327412d1fdd334327721 Mon Sep 17 00:00:00 2001 From: Prajithp Date: Thu, 17 Oct 2024 21:00:43 +0530 Subject: [PATCH 15/18] add test case for worker versioning Signed-off-by: Prajithp --- pkg/scalers/temporal.go | 4 +- tests/scalers/temporal/temporal_test.go | 77 ++++++++++++++++++++++++- 2 files changed, 77 insertions(+), 4 deletions(-) diff --git a/pkg/scalers/temporal.go b/pkg/scalers/temporal.go index c5ea4608954..0e3ab34ee05 100644 --- a/pkg/scalers/temporal.go +++ b/pkg/scalers/temporal.go @@ -40,8 +40,8 @@ type temporalMetadata struct { TaskQueue string `keda:"name=taskQueue, order=triggerMetadata;resolvedEnv"` QueueTypes []string `keda:"name=queueTypes, order=triggerMetadata, optional"` BuildID string `keda:"name=buildId, order=triggerMetadata;resolvedEnv, optional"` - AllActive bool `keda:"name=selectAllActive, order=triggerMetadata, default=true"` - Unversioned bool `keda:"name=selectUnversioned, order=triggerMetadata, default=true"` + AllActive bool `keda:"name=selectAllActive, order=triggerMetadata, default=false"` + Unversioned bool `keda:"name=selectUnversioned, order=triggerMetadata, default=false"` APIKey string `keda:"name=apiKey, order=authParams;resolvedEnv;triggerMetadata, optional"` MinConnectTimeout int `keda:"name=minConnectTimeout, order=triggerMetadata, default=5"` diff --git a/tests/scalers/temporal/temporal_test.go b/tests/scalers/temporal/temporal_test.go index 032fa7b6f6c..bf993fd87b3 100644 --- a/tests/scalers/temporal/temporal_test.go +++ b/tests/scalers/temporal/temporal_test.go @@ -31,6 +31,7 @@ var ( type templateData struct { WorkFlowCommand string WorkFlowIterations int + BuildID string DeploymentName string TestNamespace string TemporalDeploymentName string @@ -75,7 +76,7 @@ spec: containers: - name: temporal image: temporalio/admin-tools:latest - command: ["temporal", "server", "start-dev", "--ip", "0.0.0.0"] + command: ["temporal", "server", "start-dev", "--ip", "0.0.0.0", "--dynamic-config-value", "frontend.workerVersioningWorkflowAPIs=true", "--dynamic-config-value", "frontend.workerVersioningRuleAPIs=true"] ports: - containerPort: 7233 livenessProbe: @@ -114,6 +115,9 @@ spec: targetQueueSize: "2" activationTargetQueueSize: "3" endpoint: {{.TemporalDeploymentName}}.{{.TestNamespace}}.svc.cluster.local:7233 + {{- if ne .BuildID "" }} + buildId: {{.BuildID}} + {{- end}} ` deploymentTemplate = ` @@ -136,7 +140,7 @@ spec: spec: containers: - name: worker - image: "temporaliotest/omes:go-ci-latest" + image: "temporaliotest/omes:go-latest" imagePullPolicy: Always command: ["/app/temporal-omes"] args: @@ -146,6 +150,9 @@ spec: - "--run-id=test" - "--scenario=workflow_with_single_noop_activity" - "--dir-name=prepared" + {{- if ne .BuildID "" }} + - "--worker-build-id={{.BuildID}}" + {{- end}} ` jobWorkFlowTemplate = ` @@ -165,7 +172,11 @@ spec: args: - "{{.WorkFlowCommand}}" {{- if eq .WorkFlowCommand "run-scenario"}} + {{- if ne .WorkFlowIterations 0 }} - "--iterations={{.WorkFlowIterations}}" + {{ else }} + - "--duration=2m" + {{- end}} {{- end}} - "--scenario=workflow_with_single_noop_activity" - "--run-id=test" @@ -173,6 +184,33 @@ spec: restartPolicy: OnFailure backoffLimit: 10 ` + + jobUpdateBuildIDTemplate = ` +apiVersion: batch/v1 +kind: Job +metadata: + name: update-worker-version + namespace: {{.TestNamespace}} +spec: + template: + spec: + containers: + - name: workflow + image: "temporalio/admin-tools:latest" + imagePullPolicy: Always + command: ["temporal"] + args: + - "task-queue" + - "versioning" + - "commit-build-id" + - "--task-queue=workflow_with_single_noop_activity:test" + - "--build-id={{.BuildID}}" + - "--yes" + - "--force" + - "--address={{.TemporalDeploymentName}}.{{.TestNamespace}}.svc.cluster.local:7233" + restartPolicy: OnFailure + backoffLimit: 10 +` ) func getTemplateData() (templateData, []Template) { @@ -202,6 +240,7 @@ func TestTemporalScaler(t *testing.T) { testActivation(t, kc, data) testScaleOut(t, kc, data) testScaleIn(t, kc, data) + testWorkerVersioning(t, kc, data, templates) DeleteKubernetesResources(t, testNamespace, data, templates) } @@ -235,4 +274,38 @@ func testScaleIn(t *testing.T, kc *kubernetes.Clientset, data templateData) { assert.True(t, WaitForDeploymentReplicaReadyCount(t, kc, deploymentName, testNamespace, 0, 60, 5), "replica count should be %d after 5 minutes", 0) + KubectlDeleteWithTemplate(t, data, "jobWorkFlow", jobWorkFlowTemplate) +} + +func testWorkerVersioning(t *testing.T, kc *kubernetes.Clientset, data templateData, templates []Template) { + t.Log("--- testing worker versioning ---") + + data.BuildID = "1.1.1" + KubectlApplyWithTemplate(t, data, "jobUpdateBuildID", jobUpdateBuildIDTemplate) + assert.True(t, WaitForJobCount(t, kc, testNamespace, 1, 60, 3), "job update-build-id count in namespace should be 1") + assert.True(t, WaitForJobSuccess(t, kc, "update-worker-version", testNamespace, 3, 30), "job update-build-id should be successful") + KubectlDeleteWithTemplate(t, data, "jobUpdateBuildID", jobUpdateBuildIDTemplate) + + KubectlApplyMultipleWithTemplate(t, data, templates) + + data.WorkFlowIterations = 0 + KubectlApplyWithTemplate(t, data, "jobWorkFlow", jobWorkFlowTemplate) + assert.True(t, WaitForJobCount(t, kc, testNamespace, 1, 60, 3), "job count in namespace should be 1") + + assert.True(t, WaitForDeploymentReplicaReadyCount(t, kc, deploymentName, testNamespace, 1, 60, 3), + "replica count for build id %s should be %d after 3 minutes", data.BuildID, 1) + + data.BuildID = "1.1.2" + KubectlApplyWithTemplate(t, data, "jobUpdateBuildID", jobUpdateBuildIDTemplate) + assert.True(t, WaitForJobCount(t, kc, testNamespace, 2, 60, 3), "job update-build-id count in namespace should be 2") + assert.True(t, WaitForJobSuccess(t, kc, "update-worker-version", testNamespace, 3, 30), "job update-build-id should be successful") + + assert.True(t, WaitForDeploymentReplicaReadyCount(t, kc, deploymentName, testNamespace, 0, 60, 5), + "replica count for build id %s should be %d after 5 minutes", data.BuildID, 0) + + data.DeploymentName = "temporal-worker-latest" + data.ScaledObjectName = "temporal-worker-latest" + KubectlApplyMultipleWithTemplate(t, data, templates) + assert.True(t, WaitForDeploymentReplicaReadyCount(t, kc, "temporal-worker-latest", testNamespace, 1, 60, 3), + "replica count for build id %s should be %d after 3 minutes", data.BuildID, 1) } From 79af8dc6f6eb766f3212ea7f77b27d5d2d8d57d1 Mon Sep 17 00:00:00 2001 From: Prajithp Date: Fri, 1 Nov 2024 14:33:12 +0530 Subject: [PATCH 16/18] add modules to vendor Signed-off-by: Prajithp --- go.mod | 22 +- go.sum | 28 +- vendor/github.com/facebookgo/clock/LICENSE | 21 + vendor/github.com/facebookgo/clock/README.md | 104 + vendor/github.com/facebookgo/clock/clock.go | 363 + .../github.com/gogo/protobuf/jsonpb/jsonpb.go | 1435 ++ vendor/github.com/golang/mock/AUTHORS | 12 + vendor/github.com/golang/mock/CONTRIBUTORS | 37 + vendor/github.com/golang/mock/LICENSE | 202 + vendor/github.com/golang/mock/gomock/call.go | 445 + .../github.com/golang/mock/gomock/callset.go | 113 + .../golang/mock/gomock/controller.go | 336 + .../github.com/golang/mock/gomock/matchers.go | 341 + .../go-grpc-middleware/retry/backoff.go | 44 + .../go-grpc-middleware/retry/doc.go | 25 + .../go-grpc-middleware/retry/options.go | 142 + .../go-grpc-middleware/retry/retry.go | 319 + .../util/backoffutils/backoff.go | 28 + .../go-grpc-middleware/util/metautils/doc.go | 19 + .../util/metautils/nicemd.go | 126 + .../grpc-gateway/v2/runtime/BUILD.bazel | 2 +- .../grpc-gateway/v2/runtime/context.go | 11 + .../grpc-gateway/v2/runtime/errors.go | 16 +- .../grpc-gateway/v2/runtime/handler.go | 32 +- .../grpc-gateway/v2/runtime/mux.go | 69 +- vendor/github.com/nexus-rpc/sdk-go/LICENSE | 21 + .../github.com/nexus-rpc/sdk-go/nexus/api.go | 335 + .../nexus-rpc/sdk-go/nexus/client.go | 413 + .../nexus-rpc/sdk-go/nexus/completion.go | 207 + .../nexus-rpc/sdk-go/nexus/handle.go | 189 + .../nexus-rpc/sdk-go/nexus/operation.go | 344 + .../nexus-rpc/sdk-go/nexus/options.go | 65 + .../nexus-rpc/sdk-go/nexus/serializer.go | 228 + .../nexus-rpc/sdk-go/nexus/server.go | 526 + .../sdk-go/nexus/unimplemented_handler.go | 62 + vendor/github.com/pborman/uuid/.travis.yml | 10 + .../github.com/pborman/uuid/CONTRIBUTING.md | 10 + vendor/github.com/pborman/uuid/CONTRIBUTORS | 1 + vendor/github.com/pborman/uuid/LICENSE | 27 + vendor/github.com/pborman/uuid/README.md | 15 + vendor/github.com/pborman/uuid/dce.go | 84 + vendor/github.com/pborman/uuid/doc.go | 13 + vendor/github.com/pborman/uuid/hash.go | 53 + vendor/github.com/pborman/uuid/marshal.go | 85 + vendor/github.com/pborman/uuid/node.go | 50 + vendor/github.com/pborman/uuid/sql.go | 68 + vendor/github.com/pborman/uuid/time.go | 57 + vendor/github.com/pborman/uuid/util.go | 32 + vendor/github.com/pborman/uuid/uuid.go | 162 + vendor/github.com/pborman/uuid/version1.go | 23 + vendor/github.com/pborman/uuid/version4.go | 26 + vendor/github.com/robfig/cron/.gitignore | 22 + vendor/github.com/robfig/cron/.travis.yml | 1 + vendor/github.com/robfig/cron/LICENSE | 21 + vendor/github.com/robfig/cron/README.md | 6 + .../github.com/robfig/cron/constantdelay.go | 27 + vendor/github.com/robfig/cron/cron.go | 259 + vendor/github.com/robfig/cron/doc.go | 129 + vendor/github.com/robfig/cron/parser.go | 380 + vendor/github.com/robfig/cron/spec.go | 158 + vendor/go.temporal.io/api/LICENSE | 21 + .../api/batch/v1/message.go-helpers.pb.go | 250 + .../go.temporal.io/api/batch/v1/message.pb.go | 667 + .../v1/request_response.go-helpers.pb.go | 2470 +++ .../cloudservice/v1/request_response.pb.go | 5494 ++++++ .../api/cloud/cloudservice/v1/service.pb.go | 613 + .../cloud/cloudservice/v1/service.pb.gw.go | 3474 ++++ .../cloud/cloudservice/v1/service_grpc.pb.go | 1432 ++ .../identity/v1/message.go-helpers.pb.go | 509 + .../api/cloud/identity/v1/message.pb.go | 1470 ++ .../namespace/v1/message.go-helpers.pb.go | 435 + .../api/cloud/namespace/v1/message.pb.go | 1268 ++ .../operation/v1/message.go-helpers.pb.go | 65 + .../api/cloud/operation/v1/message.pb.go | 283 + .../cloud/region/v1/message.go-helpers.pb.go | 65 + .../api/cloud/region/v1/message.pb.go | 213 + .../api/command/v1/message.go-helpers.pb.go | 694 + .../api/command/v1/message.pb.go | 2709 +++ .../api/common/v1/message.go-helpers.pb.go | 620 + .../api/common/v1/message.pb.go | 1909 ++ .../api/common/v1/payload_json.go | 627 + .../enums/v1/batch_operation.go-helpers.pb.go | 70 + .../api/enums/v1/batch_operation.pb.go | 277 + .../enums/v1/command_type.go-helpers.pb.go | 62 + .../api/enums/v1/command_type.pb.go | 299 + .../api/enums/v1/common.go-helpers.pb.go | 155 + .../go.temporal.io/api/enums/v1/common.pb.go | 631 + .../api/enums/v1/event_type.go-helpers.pb.go | 99 + .../api/enums/v1/event_type.pb.go | 689 + .../enums/v1/failed_cause.go-helpers.pb.go | 182 + .../api/enums/v1/failed_cause.pb.go | 954 + .../api/enums/v1/namespace.go-helpers.pb.go | 86 + .../api/enums/v1/namespace.pb.go | 319 + .../api/enums/v1/query.go-helpers.pb.go | 67 + .../go.temporal.io/api/enums/v1/query.pb.go | 258 + .../api/enums/v1/reset.go-helpers.pb.go | 86 + .../go.temporal.io/api/enums/v1/reset.pb.go | 334 + .../api/enums/v1/schedule.go-helpers.pb.go | 51 + .../api/enums/v1/schedule.pb.go | 227 + .../api/enums/v1/task_queue.go-helpers.pb.go | 126 + .../api/enums/v1/task_queue.pb.go | 523 + .../api/enums/v1/update.go-helpers.pb.go | 66 + .../go.temporal.io/api/enums/v1/update.pb.go | 277 + .../api/enums/v1/workflow.go-helpers.pb.go | 236 + .../api/enums/v1/workflow.pb.go | 971 + .../errordetails/v1/message.go-helpers.pb.go | 620 + .../api/errordetails/v1/message.pb.go | 1354 ++ .../api/export/v1/message.go-helpers.pb.go | 102 + .../api/export/v1/message.pb.go | 252 + .../api/failure/v1/message.go-helpers.pb.go | 435 + .../api/failure/v1/message.pb.go | 1327 ++ .../api/filter/v1/message.go-helpers.pb.go | 176 + .../api/filter/v1/message.pb.go | 402 + .../api/history/v1/message.go-helpers.pb.go | 2100 +++ .../api/history/v1/message.pb.go | 9015 ++++++++++ .../api/internal/protojson/LICENSE | 27 + .../api/internal/protojson/README.md | 9 + .../api/internal/protojson/errors/errors.go | 79 + .../api/internal/protojson/genid/any_gen.go | 34 + .../api/internal/protojson/genid/api_gen.go | 106 + .../protojson/genid/descriptor_gen.go | 1087 ++ .../api/internal/protojson/genid/doc.go | 11 + .../internal/protojson/genid/duration_gen.go | 34 + .../api/internal/protojson/genid/empty_gen.go | 19 + .../protojson/genid/field_mask_gen.go | 31 + .../api/internal/protojson/genid/goname.go | 25 + .../api/internal/protojson/genid/map_entry.go | 16 + .../protojson/genid/source_context_gen.go | 31 + .../internal/protojson/genid/struct_gen.go | 116 + .../internal/protojson/genid/timestamp_gen.go | 34 + .../api/internal/protojson/genid/type_gen.go | 190 + .../api/internal/protojson/genid/wrappers.go | 13 + .../internal/protojson/genid/wrappers_gen.go | 175 + .../api/internal/protojson/json/decode.go | 340 + .../internal/protojson/json/decode_number.go | 254 + .../internal/protojson/json/decode_string.go | 91 + .../internal/protojson/json/decode_token.go | 192 + .../api/internal/protojson/json/encode.go | 267 + .../api/internal/protojson/json_marshal.go | 358 + .../api/internal/protojson/json_unmarshal.go | 723 + .../api/internal/protojson/maybe_marshal.go | 43 + .../api/internal/protojson/order/order.go | 89 + .../api/internal/protojson/order/range.go | 115 + .../api/internal/protojson/set/ints.go | 58 + .../api/internal/protojson/strs/strings.go | 190 + .../internal/protojson/strs/strings_pure.go | 28 + .../internal/protojson/strs/strings_unsafe.go | 95 + .../internal/protojson/well_known_types.go | 877 + .../api/internal/strcase/strcase.go | 139 + .../api/namespace/v1/message.go-helpers.pb.go | 250 + .../api/namespace/v1/message.pb.go | 872 + .../api/nexus/v1/message.go-helpers.pb.go | 509 + .../go.temporal.io/api/nexus/v1/message.pb.go | 1758 ++ .../v1/request_response.go-helpers.pb.go | 953 + .../operatorservice/v1/request_response.pb.go | 2023 +++ .../api/operatorservice/v1/service.pb.go | 295 + .../api/operatorservice/v1/service.pb.gw.go | 1313 ++ .../api/operatorservice/v1/service_grpc.pb.go | 624 + .../api/protocol/v1/message.go-helpers.pb.go | 65 + .../api/protocol/v1/message.pb.go | 265 + .../go.temporal.io/api/proxy/interceptor.go | 2918 +++ vendor/go.temporal.io/api/proxy/service.go | 307 + .../go.temporal.io/api/proxy/service_util.go | 60 + .../api/query/v1/message.go-helpers.pb.go | 139 + .../go.temporal.io/api/query/v1/message.pb.go | 379 + .../replication/v1/message.go-helpers.pb.go | 139 + .../api/replication/v1/message.pb.go | 358 + .../api/schedule/v1/message.go-helpers.pb.go | 620 + .../api/schedule/v1/message.pb.go | 2156 +++ .../v1/enhanced_stack_trace.go-helpers.pb.go | 213 + .../api/sdk/v1/enhanced_stack_trace.pb.go | 559 + .../task_complete_metadata.go-helpers.pb.go | 65 + .../api/sdk/v1/task_complete_metadata.pb.go | 245 + .../api/sdk/v1/user_metadata.go-helpers.pb.go | 65 + .../api/sdk/v1/user_metadata.pb.go | 203 + .../sdk/v1/workflow_metadata.go-helpers.pb.go | 139 + .../api/sdk/v1/workflow_metadata.pb.go | 389 + .../api/serviceerror/already_exists.go | 63 + .../api/serviceerror/canceled.go | 63 + .../cancellation_already_requested.go | 69 + .../client_version_not_supported.go | 84 + .../api/serviceerror/convert.go | 203 + .../api/serviceerror/data_loss.go | 63 + .../api/serviceerror/deadline_exceeded.go | 63 + .../api/serviceerror/failed_precondition.go | 63 + .../api/serviceerror/internal.go | 63 + .../api/serviceerror/invalid_argument.go | 63 + .../api/serviceerror/multi_op.go | 94 + .../api/serviceerror/multi_op_aborted.go | 64 + .../serviceerror/namespace_already_exists.go | 69 + .../serviceerror/namespace_invalid_state.go | 94 + .../api/serviceerror/namespace_not_active.go | 89 + .../api/serviceerror/namespace_not_found.go | 79 + .../api/serviceerror/newer_build_exists.go | 77 + .../api/serviceerror/not_found.go | 76 + .../api/serviceerror/permission_denied.go | 73 + .../api/serviceerror/query_failed.go | 69 + .../api/serviceerror/resource_exhausted.go | 78 + .../server_version_not_supported.go | 80 + .../api/serviceerror/serviceerror.go | 32 + .../api/serviceerror/system_workflow.go | 80 + .../api/serviceerror/unavailable.go | 63 + .../api/serviceerror/unimplemented.go | 63 + .../workflow_execution_already_started.go | 78 + .../api/serviceerror/workflow_not_ready.go | 69 + .../api/taskqueue/v1/message.go-helpers.pb.go | 731 + .../api/taskqueue/v1/message.pb.go | 1824 ++ .../api/temporalproto/deep_equal.go | 214 + .../api/temporalproto/json_marshal.go | 41 + .../api/temporalproto/json_unmarshal.go | 42 + .../api/update/v1/message.go-helpers.pb.go | 361 + .../api/update/v1/message.pb.go | 913 + .../api/version/v1/message.go-helpers.pb.go | 139 + .../api/version/v1/message.pb.go | 400 + .../api/workflow/v1/message.go-helpers.pb.go | 435 + .../api/workflow/v1/message.pb.go | 2307 +++ .../v1/request_response.go-helpers.pb.go | 4690 +++++ .../workflowservice/v1/request_response.pb.go | 15019 ++++++++++++++++ .../api/workflowservice/v1/service.pb.go | 1332 ++ .../api/workflowservice/v1/service.pb.gw.go | 10905 +++++++++++ .../api/workflowservice/v1/service_grpc.pb.go | 3160 ++++ .../v1/service_grpc.pb.mock.go | 2334 +++ vendor/go.temporal.io/sdk/LICENSE | 23 + vendor/go.temporal.io/sdk/client/client.go | 1103 ++ .../sdk/client/service_proxy.go | 52 + .../converter/byte_slice_payload_converter.go | 85 + vendor/go.temporal.io/sdk/converter/codec.go | 521 + .../sdk/converter/composite_data_converter.go | 177 + .../sdk/converter/data_converter.go | 66 + .../sdk/converter/default_data_converter.go | 46 + vendor/go.temporal.io/sdk/converter/errors.go | 54 + .../sdk/converter/failure_converter.go | 67 + .../sdk/converter/grpc_interceptor.go | 114 + .../sdk/converter/json_payload_converter.go | 69 + .../go.temporal.io/sdk/converter/metadata.go | 42 + .../sdk/converter/nil_payload_converter.go | 75 + .../sdk/converter/payload_converter.go | 72 + .../converter/proto_json_payload_converter.go | 238 + .../sdk/converter/proto_payload_converter.go | 170 + .../go.temporal.io/sdk/converter/reflect.go | 47 + vendor/go.temporal.io/sdk/converter/value.go | 51 + .../go.temporal.io/sdk/internal/activity.go | 393 + vendor/go.temporal.io/sdk/internal/client.go | 1247 ++ .../sdk/internal/common/backoff/retry.go | 196 + .../internal/common/backoff/retrypolicy.go | 210 + .../sdk/internal/common/cache/cache.go | 82 + .../sdk/internal/common/cache/lru.go | 255 + .../common/metrics/capturing_handler.go | 222 + .../sdk/internal/common/metrics/constants.go | 118 + .../sdk/internal/common/metrics/grpc.go | 131 + .../sdk/internal/common/metrics/handler.go | 142 + .../sdk/internal/common/metrics/tags.go | 161 + .../sdk/internal/common/retry/interceptor.go | 180 + .../sdk/internal/common/serializer/jsonpb.go | 61 + .../internal/common/serializer/serializer.go | 215 + .../sdk/internal/common/util/once_cell.go | 69 + .../sdk/internal/common/util/stringer.go | 195 + .../sdk/internal/common/util/util.go | 75 + vendor/go.temporal.io/sdk/internal/context.go | 364 + .../sdk/internal/encode_args.go | 138 + vendor/go.temporal.io/sdk/internal/error.go | 945 + .../sdk/internal/failure_converter.go | 289 + .../sdk/internal/grpc_dialer.go | 207 + vendor/go.temporal.io/sdk/internal/headers.go | 145 + .../sdk/internal/interceptor.go | 491 + .../sdk/internal/interceptor_base.go | 533 + .../sdk/internal/interceptor_header.go | 134 + .../sdk/internal/internal_activity.go | 434 + .../internal_command_state_machine.go | 1670 ++ .../sdk/internal/internal_eager.go | 35 + .../sdk/internal/internal_eager_activity.go | 149 + .../sdk/internal/internal_eager_workflow.go | 99 + .../sdk/internal/internal_event_handlers.go | 2078 +++ .../sdk/internal/internal_flags.go | 133 + .../sdk/internal/internal_logging_tags.go | 58 + .../sdk/internal/internal_message.go | 57 + .../internal/internal_nexus_task_handler.go | 484 + .../internal/internal_nexus_task_poller.go | 193 + .../sdk/internal/internal_nexus_worker.go | 105 + .../sdk/internal/internal_pressure_points.go | 95 + .../sdk/internal/internal_public.go | 123 + .../sdk/internal/internal_retry.go | 65 + .../sdk/internal/internal_schedule_client.go | 876 + .../internal/internal_search_attributes.go | 537 + .../sdk/internal/internal_task_handlers.go | 2341 +++ .../sdk/internal/internal_task_pollers.go | 1260 ++ .../sdk/internal/internal_time.go | 65 + .../sdk/internal/internal_update.go | 495 + .../sdk/internal/internal_utils.go | 212 + .../internal/internal_versioning_client.go | 394 + .../sdk/internal/internal_worker.go | 2009 +++ .../sdk/internal/internal_worker_base.go | 595 + .../sdk/internal/internal_worker_cache.go | 163 + .../sdk/internal/internal_workflow.go | 1962 ++ .../sdk/internal/internal_workflow_client.go | 2274 +++ .../internal/internal_workflow_testsuite.go | 2921 +++ .../sdk/internal/log/default_logger.go | 89 + .../sdk/internal/log/memory_logger.go | 106 + .../sdk/internal/log/noop_logger.go | 55 + .../sdk/internal/log/replay_logger.go | 93 + .../sdk/internal/log/test_reporter.go | 53 + .../sdk/internal/nexus_operations.go | 425 + .../sdk/internal/protocol/registry.go | 74 + .../sdk/internal/protocol/util.go | 59 + .../sdk/internal/schedule_client.go | 665 + vendor/go.temporal.io/sdk/internal/session.go | 599 + vendor/go.temporal.io/sdk/internal/tuning.go | 488 + vendor/go.temporal.io/sdk/internal/version.go | 42 + vendor/go.temporal.io/sdk/internal/worker.go | 305 + .../sdk/internal/worker_version_sets.go | 346 + .../sdk/internal/worker_versioning_rules.go | 458 + .../go.temporal.io/sdk/internal/workflow.go | 2520 +++ .../sdk/internal/workflow_deadlock.go | 219 + .../sdk/internal/workflow_testsuite.go | 1063 ++ vendor/go.temporal.io/sdk/log/logger.go | 48 + vendor/go.temporal.io/sdk/log/slog.go | 94 + vendor/go.temporal.io/sdk/log/with_logger.go | 87 + vendor/golang.org/x/sys/LICENSE | 4 +- vendor/golang.org/x/sys/cpu/cpu.go | 2 + vendor/golang.org/x/sys/cpu/cpu_arm64.go | 12 + .../golang.org/x/sys/cpu/cpu_linux_arm64.go | 5 + vendor/golang.org/x/sys/unix/mkerrors.sh | 1 + .../golang.org/x/sys/unix/syscall_darwin.go | 12 + vendor/golang.org/x/sys/unix/syscall_linux.go | 1 + .../golang.org/x/sys/unix/syscall_openbsd.go | 1 + .../x/sys/unix/zerrors_darwin_amd64.go | 5 + .../x/sys/unix/zerrors_darwin_arm64.go | 5 + vendor/golang.org/x/sys/unix/zerrors_linux.go | 38 +- .../x/sys/unix/zerrors_linux_386.go | 2 + .../x/sys/unix/zerrors_linux_amd64.go | 2 + .../x/sys/unix/zerrors_linux_arm.go | 2 + .../x/sys/unix/zerrors_linux_arm64.go | 2 + .../x/sys/unix/zerrors_linux_loong64.go | 2 + .../x/sys/unix/zerrors_linux_mips.go | 2 + .../x/sys/unix/zerrors_linux_mips64.go | 2 + .../x/sys/unix/zerrors_linux_mips64le.go | 2 + .../x/sys/unix/zerrors_linux_mipsle.go | 2 + .../x/sys/unix/zerrors_linux_ppc.go | 2 + .../x/sys/unix/zerrors_linux_ppc64.go | 2 + .../x/sys/unix/zerrors_linux_ppc64le.go | 2 + .../x/sys/unix/zerrors_linux_riscv64.go | 2 + .../x/sys/unix/zerrors_linux_s390x.go | 2 + .../x/sys/unix/zerrors_linux_sparc64.go | 2 + .../x/sys/unix/zsyscall_darwin_amd64.go | 48 + .../x/sys/unix/zsyscall_darwin_amd64.s | 10 + .../x/sys/unix/zsyscall_darwin_arm64.go | 48 + .../x/sys/unix/zsyscall_darwin_arm64.s | 10 + .../golang.org/x/sys/unix/zsyscall_linux.go | 16 + .../x/sys/unix/zsyscall_openbsd_386.go | 24 + .../x/sys/unix/zsyscall_openbsd_386.s | 5 + .../x/sys/unix/zsyscall_openbsd_amd64.go | 24 + .../x/sys/unix/zsyscall_openbsd_amd64.s | 5 + .../x/sys/unix/zsyscall_openbsd_arm.go | 24 + .../x/sys/unix/zsyscall_openbsd_arm.s | 5 + .../x/sys/unix/zsyscall_openbsd_arm64.go | 24 + .../x/sys/unix/zsyscall_openbsd_arm64.s | 5 + .../x/sys/unix/zsyscall_openbsd_mips64.go | 24 + .../x/sys/unix/zsyscall_openbsd_mips64.s | 5 + .../x/sys/unix/zsyscall_openbsd_ppc64.go | 24 + .../x/sys/unix/zsyscall_openbsd_ppc64.s | 6 + .../x/sys/unix/zsyscall_openbsd_riscv64.go | 24 + .../x/sys/unix/zsyscall_openbsd_riscv64.s | 5 + .../x/sys/unix/zsysnum_linux_386.go | 1 + .../x/sys/unix/zsysnum_linux_amd64.go | 1 + .../x/sys/unix/zsysnum_linux_arm.go | 1 + .../x/sys/unix/zsysnum_linux_arm64.go | 1 + .../x/sys/unix/zsysnum_linux_loong64.go | 1 + .../x/sys/unix/zsysnum_linux_mips.go | 1 + .../x/sys/unix/zsysnum_linux_mips64.go | 1 + .../x/sys/unix/zsysnum_linux_mips64le.go | 1 + .../x/sys/unix/zsysnum_linux_mipsle.go | 1 + .../x/sys/unix/zsysnum_linux_ppc.go | 1 + .../x/sys/unix/zsysnum_linux_ppc64.go | 1 + .../x/sys/unix/zsysnum_linux_ppc64le.go | 1 + .../x/sys/unix/zsysnum_linux_riscv64.go | 1 + .../x/sys/unix/zsysnum_linux_s390x.go | 1 + .../x/sys/unix/zsysnum_linux_sparc64.go | 1 + vendor/golang.org/x/sys/unix/ztypes_linux.go | 10 +- .../x/sys/windows/security_windows.go | 2 +- .../x/sys/windows/syscall_windows.go | 12 +- .../golang.org/x/sys/windows/types_windows.go | 71 +- .../x/sys/windows/zsyscall_windows.go | 49 +- .../googleapis/api/expr/v1alpha1/syntax.pb.go | 298 +- vendor/modules.txt | 92 +- 384 files changed, 168117 insertions(+), 189 deletions(-) create mode 100644 vendor/github.com/facebookgo/clock/LICENSE create mode 100644 vendor/github.com/facebookgo/clock/README.md create mode 100644 vendor/github.com/facebookgo/clock/clock.go create mode 100644 vendor/github.com/gogo/protobuf/jsonpb/jsonpb.go create mode 100644 vendor/github.com/golang/mock/AUTHORS create mode 100644 vendor/github.com/golang/mock/CONTRIBUTORS create mode 100644 vendor/github.com/golang/mock/LICENSE create mode 100644 vendor/github.com/golang/mock/gomock/call.go create mode 100644 vendor/github.com/golang/mock/gomock/callset.go create mode 100644 vendor/github.com/golang/mock/gomock/controller.go create mode 100644 vendor/github.com/golang/mock/gomock/matchers.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/backoff.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/doc.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/options.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/retry.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils/backoff.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/doc.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/nicemd.go create mode 100644 vendor/github.com/nexus-rpc/sdk-go/LICENSE create mode 100644 vendor/github.com/nexus-rpc/sdk-go/nexus/api.go create mode 100644 vendor/github.com/nexus-rpc/sdk-go/nexus/client.go create mode 100644 vendor/github.com/nexus-rpc/sdk-go/nexus/completion.go create mode 100644 vendor/github.com/nexus-rpc/sdk-go/nexus/handle.go create mode 100644 vendor/github.com/nexus-rpc/sdk-go/nexus/operation.go create mode 100644 vendor/github.com/nexus-rpc/sdk-go/nexus/options.go create mode 100644 vendor/github.com/nexus-rpc/sdk-go/nexus/serializer.go create mode 100644 vendor/github.com/nexus-rpc/sdk-go/nexus/server.go create mode 100644 vendor/github.com/nexus-rpc/sdk-go/nexus/unimplemented_handler.go create mode 100644 vendor/github.com/pborman/uuid/.travis.yml create mode 100644 vendor/github.com/pborman/uuid/CONTRIBUTING.md create mode 100644 vendor/github.com/pborman/uuid/CONTRIBUTORS create mode 100644 vendor/github.com/pborman/uuid/LICENSE create mode 100644 vendor/github.com/pborman/uuid/README.md create mode 100644 vendor/github.com/pborman/uuid/dce.go create mode 100644 vendor/github.com/pborman/uuid/doc.go create mode 100644 vendor/github.com/pborman/uuid/hash.go create mode 100644 vendor/github.com/pborman/uuid/marshal.go create mode 100644 vendor/github.com/pborman/uuid/node.go create mode 100644 vendor/github.com/pborman/uuid/sql.go create mode 100644 vendor/github.com/pborman/uuid/time.go create mode 100644 vendor/github.com/pborman/uuid/util.go create mode 100644 vendor/github.com/pborman/uuid/uuid.go create mode 100644 vendor/github.com/pborman/uuid/version1.go create mode 100644 vendor/github.com/pborman/uuid/version4.go create mode 100644 vendor/github.com/robfig/cron/.gitignore create mode 100644 vendor/github.com/robfig/cron/.travis.yml create mode 100644 vendor/github.com/robfig/cron/LICENSE create mode 100644 vendor/github.com/robfig/cron/README.md create mode 100644 vendor/github.com/robfig/cron/constantdelay.go create mode 100644 vendor/github.com/robfig/cron/cron.go create mode 100644 vendor/github.com/robfig/cron/doc.go create mode 100644 vendor/github.com/robfig/cron/parser.go create mode 100644 vendor/github.com/robfig/cron/spec.go create mode 100644 vendor/go.temporal.io/api/LICENSE create mode 100644 vendor/go.temporal.io/api/batch/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/batch/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/cloud/cloudservice/v1/request_response.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/cloud/cloudservice/v1/request_response.pb.go create mode 100644 vendor/go.temporal.io/api/cloud/cloudservice/v1/service.pb.go create mode 100644 vendor/go.temporal.io/api/cloud/cloudservice/v1/service.pb.gw.go create mode 100644 vendor/go.temporal.io/api/cloud/cloudservice/v1/service_grpc.pb.go create mode 100644 vendor/go.temporal.io/api/cloud/identity/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/cloud/identity/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/cloud/namespace/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/cloud/namespace/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/cloud/operation/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/cloud/operation/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/cloud/region/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/cloud/region/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/command/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/command/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/common/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/common/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/common/v1/payload_json.go create mode 100644 vendor/go.temporal.io/api/enums/v1/batch_operation.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/batch_operation.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/command_type.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/command_type.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/common.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/common.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/event_type.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/event_type.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/failed_cause.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/failed_cause.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/namespace.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/namespace.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/query.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/query.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/reset.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/reset.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/schedule.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/schedule.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/task_queue.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/task_queue.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/update.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/update.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/workflow.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/enums/v1/workflow.pb.go create mode 100644 vendor/go.temporal.io/api/errordetails/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/errordetails/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/export/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/export/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/failure/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/failure/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/filter/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/filter/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/history/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/history/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/LICENSE create mode 100644 vendor/go.temporal.io/api/internal/protojson/README.md create mode 100644 vendor/go.temporal.io/api/internal/protojson/errors/errors.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/any_gen.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/api_gen.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/descriptor_gen.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/doc.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/duration_gen.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/empty_gen.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/field_mask_gen.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/goname.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/map_entry.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/source_context_gen.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/struct_gen.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/timestamp_gen.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/type_gen.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/wrappers.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/genid/wrappers_gen.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/json/decode.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/json/decode_number.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/json/decode_string.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/json/decode_token.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/json/encode.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/json_marshal.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/json_unmarshal.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/maybe_marshal.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/order/order.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/order/range.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/set/ints.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/strs/strings.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/strs/strings_pure.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/strs/strings_unsafe.go create mode 100644 vendor/go.temporal.io/api/internal/protojson/well_known_types.go create mode 100644 vendor/go.temporal.io/api/internal/strcase/strcase.go create mode 100644 vendor/go.temporal.io/api/namespace/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/namespace/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/nexus/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/nexus/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/operatorservice/v1/request_response.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/operatorservice/v1/request_response.pb.go create mode 100644 vendor/go.temporal.io/api/operatorservice/v1/service.pb.go create mode 100644 vendor/go.temporal.io/api/operatorservice/v1/service.pb.gw.go create mode 100644 vendor/go.temporal.io/api/operatorservice/v1/service_grpc.pb.go create mode 100644 vendor/go.temporal.io/api/protocol/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/protocol/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/proxy/interceptor.go create mode 100644 vendor/go.temporal.io/api/proxy/service.go create mode 100644 vendor/go.temporal.io/api/proxy/service_util.go create mode 100644 vendor/go.temporal.io/api/query/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/query/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/replication/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/replication/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/schedule/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/schedule/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/sdk/v1/enhanced_stack_trace.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/sdk/v1/enhanced_stack_trace.pb.go create mode 100644 vendor/go.temporal.io/api/sdk/v1/task_complete_metadata.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/sdk/v1/task_complete_metadata.pb.go create mode 100644 vendor/go.temporal.io/api/sdk/v1/user_metadata.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/sdk/v1/user_metadata.pb.go create mode 100644 vendor/go.temporal.io/api/sdk/v1/workflow_metadata.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/sdk/v1/workflow_metadata.pb.go create mode 100644 vendor/go.temporal.io/api/serviceerror/already_exists.go create mode 100644 vendor/go.temporal.io/api/serviceerror/canceled.go create mode 100644 vendor/go.temporal.io/api/serviceerror/cancellation_already_requested.go create mode 100644 vendor/go.temporal.io/api/serviceerror/client_version_not_supported.go create mode 100644 vendor/go.temporal.io/api/serviceerror/convert.go create mode 100644 vendor/go.temporal.io/api/serviceerror/data_loss.go create mode 100644 vendor/go.temporal.io/api/serviceerror/deadline_exceeded.go create mode 100644 vendor/go.temporal.io/api/serviceerror/failed_precondition.go create mode 100644 vendor/go.temporal.io/api/serviceerror/internal.go create mode 100644 vendor/go.temporal.io/api/serviceerror/invalid_argument.go create mode 100644 vendor/go.temporal.io/api/serviceerror/multi_op.go create mode 100644 vendor/go.temporal.io/api/serviceerror/multi_op_aborted.go create mode 100644 vendor/go.temporal.io/api/serviceerror/namespace_already_exists.go create mode 100644 vendor/go.temporal.io/api/serviceerror/namespace_invalid_state.go create mode 100644 vendor/go.temporal.io/api/serviceerror/namespace_not_active.go create mode 100644 vendor/go.temporal.io/api/serviceerror/namespace_not_found.go create mode 100644 vendor/go.temporal.io/api/serviceerror/newer_build_exists.go create mode 100644 vendor/go.temporal.io/api/serviceerror/not_found.go create mode 100644 vendor/go.temporal.io/api/serviceerror/permission_denied.go create mode 100644 vendor/go.temporal.io/api/serviceerror/query_failed.go create mode 100644 vendor/go.temporal.io/api/serviceerror/resource_exhausted.go create mode 100644 vendor/go.temporal.io/api/serviceerror/server_version_not_supported.go create mode 100644 vendor/go.temporal.io/api/serviceerror/serviceerror.go create mode 100644 vendor/go.temporal.io/api/serviceerror/system_workflow.go create mode 100644 vendor/go.temporal.io/api/serviceerror/unavailable.go create mode 100644 vendor/go.temporal.io/api/serviceerror/unimplemented.go create mode 100644 vendor/go.temporal.io/api/serviceerror/workflow_execution_already_started.go create mode 100644 vendor/go.temporal.io/api/serviceerror/workflow_not_ready.go create mode 100644 vendor/go.temporal.io/api/taskqueue/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/taskqueue/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/temporalproto/deep_equal.go create mode 100644 vendor/go.temporal.io/api/temporalproto/json_marshal.go create mode 100644 vendor/go.temporal.io/api/temporalproto/json_unmarshal.go create mode 100644 vendor/go.temporal.io/api/update/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/update/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/version/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/version/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/workflow/v1/message.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/workflow/v1/message.pb.go create mode 100644 vendor/go.temporal.io/api/workflowservice/v1/request_response.go-helpers.pb.go create mode 100644 vendor/go.temporal.io/api/workflowservice/v1/request_response.pb.go create mode 100644 vendor/go.temporal.io/api/workflowservice/v1/service.pb.go create mode 100644 vendor/go.temporal.io/api/workflowservice/v1/service.pb.gw.go create mode 100644 vendor/go.temporal.io/api/workflowservice/v1/service_grpc.pb.go create mode 100644 vendor/go.temporal.io/api/workflowservicemock/v1/service_grpc.pb.mock.go create mode 100644 vendor/go.temporal.io/sdk/LICENSE create mode 100644 vendor/go.temporal.io/sdk/client/client.go create mode 100644 vendor/go.temporal.io/sdk/client/service_proxy.go create mode 100644 vendor/go.temporal.io/sdk/converter/byte_slice_payload_converter.go create mode 100644 vendor/go.temporal.io/sdk/converter/codec.go create mode 100644 vendor/go.temporal.io/sdk/converter/composite_data_converter.go create mode 100644 vendor/go.temporal.io/sdk/converter/data_converter.go create mode 100644 vendor/go.temporal.io/sdk/converter/default_data_converter.go create mode 100644 vendor/go.temporal.io/sdk/converter/errors.go create mode 100644 vendor/go.temporal.io/sdk/converter/failure_converter.go create mode 100644 vendor/go.temporal.io/sdk/converter/grpc_interceptor.go create mode 100644 vendor/go.temporal.io/sdk/converter/json_payload_converter.go create mode 100644 vendor/go.temporal.io/sdk/converter/metadata.go create mode 100644 vendor/go.temporal.io/sdk/converter/nil_payload_converter.go create mode 100644 vendor/go.temporal.io/sdk/converter/payload_converter.go create mode 100644 vendor/go.temporal.io/sdk/converter/proto_json_payload_converter.go create mode 100644 vendor/go.temporal.io/sdk/converter/proto_payload_converter.go create mode 100644 vendor/go.temporal.io/sdk/converter/reflect.go create mode 100644 vendor/go.temporal.io/sdk/converter/value.go create mode 100644 vendor/go.temporal.io/sdk/internal/activity.go create mode 100644 vendor/go.temporal.io/sdk/internal/client.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/backoff/retry.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/backoff/retrypolicy.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/cache/cache.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/cache/lru.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/metrics/capturing_handler.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/metrics/constants.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/metrics/grpc.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/metrics/handler.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/metrics/tags.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/retry/interceptor.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/serializer/jsonpb.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/serializer/serializer.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/util/once_cell.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/util/stringer.go create mode 100644 vendor/go.temporal.io/sdk/internal/common/util/util.go create mode 100644 vendor/go.temporal.io/sdk/internal/context.go create mode 100644 vendor/go.temporal.io/sdk/internal/encode_args.go create mode 100644 vendor/go.temporal.io/sdk/internal/error.go create mode 100644 vendor/go.temporal.io/sdk/internal/failure_converter.go create mode 100644 vendor/go.temporal.io/sdk/internal/grpc_dialer.go create mode 100644 vendor/go.temporal.io/sdk/internal/headers.go create mode 100644 vendor/go.temporal.io/sdk/internal/interceptor.go create mode 100644 vendor/go.temporal.io/sdk/internal/interceptor_base.go create mode 100644 vendor/go.temporal.io/sdk/internal/interceptor_header.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_activity.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_command_state_machine.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_eager.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_eager_activity.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_eager_workflow.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_event_handlers.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_flags.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_logging_tags.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_message.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_nexus_task_handler.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_nexus_task_poller.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_nexus_worker.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_pressure_points.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_public.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_retry.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_schedule_client.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_search_attributes.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_task_handlers.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_task_pollers.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_time.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_update.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_utils.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_versioning_client.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_worker.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_worker_base.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_worker_cache.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_workflow.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_workflow_client.go create mode 100644 vendor/go.temporal.io/sdk/internal/internal_workflow_testsuite.go create mode 100644 vendor/go.temporal.io/sdk/internal/log/default_logger.go create mode 100644 vendor/go.temporal.io/sdk/internal/log/memory_logger.go create mode 100644 vendor/go.temporal.io/sdk/internal/log/noop_logger.go create mode 100644 vendor/go.temporal.io/sdk/internal/log/replay_logger.go create mode 100644 vendor/go.temporal.io/sdk/internal/log/test_reporter.go create mode 100644 vendor/go.temporal.io/sdk/internal/nexus_operations.go create mode 100644 vendor/go.temporal.io/sdk/internal/protocol/registry.go create mode 100644 vendor/go.temporal.io/sdk/internal/protocol/util.go create mode 100644 vendor/go.temporal.io/sdk/internal/schedule_client.go create mode 100644 vendor/go.temporal.io/sdk/internal/session.go create mode 100644 vendor/go.temporal.io/sdk/internal/tuning.go create mode 100644 vendor/go.temporal.io/sdk/internal/version.go create mode 100644 vendor/go.temporal.io/sdk/internal/worker.go create mode 100644 vendor/go.temporal.io/sdk/internal/worker_version_sets.go create mode 100644 vendor/go.temporal.io/sdk/internal/worker_versioning_rules.go create mode 100644 vendor/go.temporal.io/sdk/internal/workflow.go create mode 100644 vendor/go.temporal.io/sdk/internal/workflow_deadlock.go create mode 100644 vendor/go.temporal.io/sdk/internal/workflow_testsuite.go create mode 100644 vendor/go.temporal.io/sdk/log/logger.go create mode 100644 vendor/go.temporal.io/sdk/log/slog.go create mode 100644 vendor/go.temporal.io/sdk/log/with_logger.go diff --git a/go.mod b/go.mod index 4322a6cafee..cdbedb2b282 100644 --- a/go.mod +++ b/go.mod @@ -94,6 +94,7 @@ require ( go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v1.28.0 go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v1.28.0 go.opentelemetry.io/otel/metric v1.28.0 + go.temporal.io/sdk v1.29.1 go.uber.org/mock v0.4.0 golang.org/x/oauth2 v0.22.0 golang.org/x/sync v0.8.0 @@ -119,6 +120,15 @@ require ( sigs.k8s.io/kustomize/kustomize/v5 v5.4.3 ) +require ( + github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a // indirect + github.com/golang/mock v1.6.0 // indirect + github.com/nexus-rpc/sdk-go v0.0.10 // indirect + github.com/pborman/uuid v1.2.1 // indirect + github.com/robfig/cron v1.2.0 // indirect + go.temporal.io/api v1.38.0 // indirect +) + // Remove this when they merge the PR and cut a release https://github.com/open-policy-agent/cert-controller/pull/202 replace github.com/open-policy-agent/cert-controller => github.com/jorturfer/cert-controller v0.0.0-20240427003941-363ba56751d7 @@ -261,7 +271,7 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.1.0 // indirect github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 // indirect - github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 // indirect + github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed // indirect github.com/hashicorp/errwrap v1.1.0 // indirect github.com/hashicorp/go-cleanhttp v0.5.2 // indirect @@ -354,17 +364,17 @@ require ( golang.org/x/crypto v0.25.0 golang.org/x/exp v0.0.0-20240719175910-8a7402abbf56 golang.org/x/mod v0.19.0 // indirect - golang.org/x/net v0.27.0 // indirect - golang.org/x/sys v0.22.0 // indirect + golang.org/x/net v0.28.0 // indirect + golang.org/x/sys v0.24.0 // indirect golang.org/x/term v0.22.0 // indirect - golang.org/x/text v0.16.0 // indirect + golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.23.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect gomodules.xyz/jsonpatch/v2 v2.4.0 // indirect google.golang.org/genproto v0.0.0-20240730163845-b1a4ccb954bf // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240725223205-93522f1f2a9f // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240730163845-b1a4ccb954bf // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect gopkg.in/evanphx/json-patch.v4 v4.12.0 // indirect gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect diff --git a/go.sum b/go.sum index e7cefad3489..168e7d5a74e 100644 --- a/go.sum +++ b/go.sum @@ -1062,6 +1062,8 @@ github.com/evanphx/json-patch/v5 v5.9.0 h1:kcBlZQbplgElYIlo/n1hJbls2z/1awpXxpRi0 github.com/evanphx/json-patch/v5 v5.9.0/go.mod h1:VNkHZ/282BpEyt/tObQO8s5CMPmYYq14uClGH4abBuQ= github.com/expr-lang/expr v1.16.9 h1:WUAzmR0JNI9JCiF0/ewwHB1gmcGw5wW7nWt8gc6PpCI= github.com/expr-lang/expr v1.16.9/go.mod h1:8/vRC7+7HBzESEqt5kKpYXxrxkr31SaO8r40VO/1IT4= +github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= +github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fatih/color v1.16.0 h1:zmkK9Ngbjj+K0yRhTVONQh1p/HknKYSlNT+vZCzyokM= github.com/fatih/color v1.16.0/go.mod h1:fL2Sau1YI5c0pdGEVCbKQbLXB6edEj1ZgiY4NijnWvE= @@ -1277,6 +1279,7 @@ github.com/google/s2a-go v0.1.8 h1:zZDs9gcbt9ZPLV0ndSyQk6Kacx2g/X+SKYovpnz3SMM= github.com/google/s2a-go v0.1.8/go.mod h1:6iNWHTpQ+nfNRN5E00MSdfDwVesa8hhS32PhPO8deJA= github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 h1:El6M4kTTCOh6aBiKaUGG7oYTSPP8MxqL4YI3kZKwcP4= github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510/go.mod h1:pupxD2MaaD3pAXIBCelhxNneeOaAeabZDe5s4K6zSpQ= +github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= @@ -1335,8 +1338,8 @@ github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFb github.com/grpc-ecosystem/grpc-gateway/v2 v2.7.0/go.mod h1:hgWBS7lorOAVIJEQMi4ZsPv9hVvWI6+ch50m39Pf2Ks= github.com/grpc-ecosystem/grpc-gateway/v2 v2.11.3/go.mod h1:o//XUCC/F+yRGJoPO/VU0GSB0f8Nhgmxx0VIRUvaC0w= github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0/go.mod h1:YN5jB8ie0yfIUg6VvR9Kz84aCaG7AsGZnLjhHbUqwPg= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 h1:bkypFPDjIYGfCYD5mRBvpqxfYX1YCS1PXdKYWi8FsN0= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0/go.mod h1:P+Lt/0by1T8bfcF3z737NnSbmxQAppXMRziHUxPOC8k= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed h1:5upAirOpQc1Q53c0bnx2ufif5kANL7bfZWcc6VJWJd8= github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed/go.mod h1:tMWxXQ9wFIaZeTI9F+hmhFiGpFmhOHzyShyFUhRm0H4= github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q= @@ -1546,6 +1549,8 @@ github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJE github.com/neelance/sourcemap v0.0.0-20200213170602-2833bce08e4c/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM= github.com/newrelic/newrelic-client-go v1.1.0 h1:aflNjzQ21c+2GwBVh+UbAf9lznkRfCcVABoc5UM4IXw= github.com/newrelic/newrelic-client-go v1.1.0/go.mod h1:RYMXt7hgYw7nzuXIGd2BH0F1AivgWw7WrBhNBQZEB4k= +github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= +github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/oapi-codegen/runtime v1.1.1 h1:EXLHh0DXIJnWhdRPN2w4MXAzFyE4CskzhNLUmtpMYro= @@ -1573,6 +1578,8 @@ github.com/otiai10/curr v1.0.0/go.mod h1:LskTG5wDwr8Rs+nNQ+1LlxRjAtTZZjtJW4rMXl6 github.com/otiai10/mint v1.3.0/go.mod h1:F5AjcsTsWUqX+Na9fpHb52P8pcRX2CI6A3ctIT91xUo= github.com/otiai10/mint v1.3.3/go.mod h1:/yxELlJQ0ufhjUwhshSj+wFjZ78CnZ48/1wtmBH1OTc= github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= +github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= +github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/phayes/freeport v0.0.0-20220201140144-74d24b5ae9f5 h1:Ii+DKncOVM8Cu1Hc+ETb5K+23HdAMvESYE3ZJ5b5cMI= github.com/phayes/freeport v0.0.0-20220201140144-74d24b5ae9f5/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE= @@ -1622,6 +1629,8 @@ github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec/go.mod h1:qq github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.4.4 h1:8TfxU8dW6PdqD27gjM8MVNuicgxIjxpm4K7x4jp8sis= github.com/rivo/uniseg v0.4.4/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= +github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= +github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= github.com/robfig/cron/v3 v3.0.1 h1:WdRxkvbJztn8LMz/QEvLN5sBU+xKpSqwwUO1Pjr4qDs= github.com/robfig/cron/v3 v3.0.1/go.mod h1:eQICP3HwyT7UooqI/z+Ov+PtYAWygg1TEWWzGIFLtro= github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= @@ -1816,6 +1825,10 @@ go.opentelemetry.io/proto/otlp v1.3.1 h1:TrMUixzpM0yuc/znrFTP9MMRh8trP93mkCiDVeX go.opentelemetry.io/proto/otlp v1.3.1/go.mod h1:0X1WI4de4ZsLrrJNLAQbFeLCm3T7yBkR0XqQ7niQU+8= go.starlark.net v0.0.0-20231121155337-90ade8b19d09 h1:hzy3LFnSN8kuQK8h9tHl4ndF6UruMj47OqwqsS+/Ai4= go.starlark.net v0.0.0-20231121155337-90ade8b19d09/go.mod h1:LcLNIzVOMp4oV+uusnpk+VU+SzXaJakUuBjoCSWH5dM= +go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= +go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= +go.temporal.io/sdk v1.29.1 h1:y+sUMbUhTU9rj50mwIZAPmcXCtgUdOWS9xHDYRYSgZ0= +go.temporal.io/sdk v1.29.1/go.mod h1:kp//DRvn3CqQVBCtjL51Oicp9wrZYB2s6row1UgzcKQ= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= @@ -2048,8 +2061,9 @@ golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= +golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/telemetry v0.0.0-20240228155512-f48c80bd79b2/go.mod h1:TeRTkGYfJXctD9OcfyVLyj2J3IxLnKwHJR8f4D8a3YE= golang.org/x/term v0.22.0 h1:BbsgPEJULsl2fV/AT3v15Mjva5yXKQDyKf+TbDz7QJk= golang.org/x/term v0.22.0/go.mod h1:F3qCibpT5AMpCRfhfT53vVJwhLtIVHhB9XDjfFvnMI4= @@ -2380,8 +2394,8 @@ google.golang.org/genproto/googleapis/api v0.0.0-20230711160842-782d3b101e98/go. google.golang.org/genproto/googleapis/api v0.0.0-20230726155614-23370e0ffb3e/go.mod h1:rsr7RhLuwsDKL7RmgDDCUc6yaGr1iqceVb5Wv6f6YvQ= google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d/go.mod h1:KjSP20unUpOx5kyQUFa7k4OJg0qeJ7DEZflGDu2p6Bk= google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157/go.mod h1:99sLkeliLXfdj2J75X3Ho+rrVCaJze0uwN7zDDkjPVU= -google.golang.org/genproto/googleapis/api v0.0.0-20240725223205-93522f1f2a9f h1:b1Ln/PG8orm0SsBbHZWke8dDp2lrCD4jSmfglFpTZbk= -google.golang.org/genproto/googleapis/api v0.0.0-20240725223205-93522f1f2a9f/go.mod h1:AHT0dDg3SoMOgZGnZk29b5xTbPHMoEC8qthmBLJCpys= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= google.golang.org/genproto/googleapis/bytestream v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:ylj+BE99M198VPbBh6A8d9n3w8fChvyLK3wwBOjXBFA= google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234015-3fc162c6f38a/go.mod h1:xURIpW9ES5+/GZhnV6beoEtxQrnkRGIfP5VQG2tCBLc= google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= @@ -2395,8 +2409,8 @@ google.golang.org/genproto/googleapis/rpc v0.0.0-20230803162519-f966b187b2e5/go. google.golang.org/genproto/googleapis/rpc v0.0.0-20240318140521-94a12d6c2237/go.mod h1:WtryC6hu0hhx87FDGxWCDptyssuo68sk10vYjF+T9fY= google.golang.org/genproto/googleapis/rpc v0.0.0-20240521202816-d264139d666e/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240730163845-b1a4ccb954bf h1:liao9UHurZLtiEwBgT9LMOnKYsHze6eA6w1KQCMVN2Q= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240730163845-b1a4ccb954bf/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= diff --git a/vendor/github.com/facebookgo/clock/LICENSE b/vendor/github.com/facebookgo/clock/LICENSE new file mode 100644 index 00000000000..ce212cb1cee --- /dev/null +++ b/vendor/github.com/facebookgo/clock/LICENSE @@ -0,0 +1,21 @@ +The MIT License (MIT) + +Copyright (c) 2014 Ben Johnson + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/vendor/github.com/facebookgo/clock/README.md b/vendor/github.com/facebookgo/clock/README.md new file mode 100644 index 00000000000..5d4f4fe72e7 --- /dev/null +++ b/vendor/github.com/facebookgo/clock/README.md @@ -0,0 +1,104 @@ +clock [![Build Status](https://drone.io/github.com/benbjohnson/clock/status.png)](https://drone.io/github.com/benbjohnson/clock/latest) [![Coverage Status](https://coveralls.io/repos/benbjohnson/clock/badge.png?branch=master)](https://coveralls.io/r/benbjohnson/clock?branch=master) [![GoDoc](https://godoc.org/github.com/benbjohnson/clock?status.png)](https://godoc.org/github.com/benbjohnson/clock) ![Project status](http://img.shields.io/status/experimental.png?color=red) +===== + +Clock is a small library for mocking time in Go. It provides an interface +around the standard library's [`time`][time] package so that the application +can use the realtime clock while tests can use the mock clock. + +[time]: http://golang.org/pkg/time/ + + +## Usage + +### Realtime Clock + +Your application can maintain a `Clock` variable that will allow realtime and +mock clocks to be interchangable. For example, if you had an `Application` type: + +```go +import "github.com/benbjohnson/clock" + +type Application struct { + Clock clock.Clock +} +``` + +You could initialize it to use the realtime clock like this: + +```go +var app Application +app.Clock = clock.New() +... +``` + +Then all timers and time-related functionality should be performed from the +`Clock` variable. + + +### Mocking time + +In your tests, you will want to use a `Mock` clock: + +```go +import ( + "testing" + + "github.com/benbjohnson/clock" +) + +func TestApplication_DoSomething(t *testing.T) { + mock := clock.NewMock() + app := Application{Clock: mock} + ... +} +``` + +Now that you've initialized your application to use the mock clock, you can +adjust the time programmatically. The mock clock always starts from the Unix +epoch (midnight, Jan 1, 1970 UTC). + + +### Controlling time + +The mock clock provides the same functions that the standard library's `time` +package provides. For example, to find the current time, you use the `Now()` +function: + +```go +mock := clock.NewMock() + +// Find the current time. +mock.Now().UTC() // 1970-01-01 00:00:00 +0000 UTC + +// Move the clock forward. +mock.Add(2 * time.Hour) + +// Check the time again. It's 2 hours later! +mock.Now().UTC() // 1970-01-01 02:00:00 +0000 UTC +``` + +Timers and Tickers are also controlled by this same mock clock. They will only +execute when the clock is moved forward: + +``` +mock := clock.NewMock() +count := 0 + +// Kick off a timer to increment every 1 mock second. +go func() { + ticker := clock.Ticker(1 * time.Second) + for { + <-ticker.C + count++ + } +}() +runtime.Gosched() + +// Move the clock forward 10 second. +mock.Add(10 * time.Second) + +// This prints 10. +fmt.Println(count) +``` + + diff --git a/vendor/github.com/facebookgo/clock/clock.go b/vendor/github.com/facebookgo/clock/clock.go new file mode 100644 index 00000000000..bca1a7ba8b3 --- /dev/null +++ b/vendor/github.com/facebookgo/clock/clock.go @@ -0,0 +1,363 @@ +package clock + +import ( + "runtime" + "sort" + "sync" + "time" +) + +// Clock represents an interface to the functions in the standard library time +// package. Two implementations are available in the clock package. The first +// is a real-time clock which simply wraps the time package's functions. The +// second is a mock clock which will only make forward progress when +// programmatically adjusted. +type Clock interface { + After(d time.Duration) <-chan time.Time + AfterFunc(d time.Duration, f func()) *Timer + Now() time.Time + Sleep(d time.Duration) + Tick(d time.Duration) <-chan time.Time + Ticker(d time.Duration) *Ticker + Timer(d time.Duration) *Timer +} + +// New returns an instance of a real-time clock. +func New() Clock { + return &clock{} +} + +// clock implements a real-time clock by simply wrapping the time package functions. +type clock struct{} + +func (c *clock) After(d time.Duration) <-chan time.Time { return time.After(d) } + +func (c *clock) AfterFunc(d time.Duration, f func()) *Timer { + return &Timer{timer: time.AfterFunc(d, f)} +} + +func (c *clock) Now() time.Time { return time.Now() } + +func (c *clock) Sleep(d time.Duration) { time.Sleep(d) } + +func (c *clock) Tick(d time.Duration) <-chan time.Time { return time.Tick(d) } + +func (c *clock) Ticker(d time.Duration) *Ticker { + t := time.NewTicker(d) + return &Ticker{C: t.C, ticker: t} +} + +func (c *clock) Timer(d time.Duration) *Timer { + t := time.NewTimer(d) + return &Timer{C: t.C, timer: t} +} + +// Mock represents a mock clock that only moves forward programmically. +// It can be preferable to a real-time clock when testing time-based functionality. +type Mock struct { + mu sync.Mutex + now time.Time // current time + timers clockTimers // tickers & timers + + calls Calls + waiting []waiting + callsMutex sync.Mutex +} + +// NewMock returns an instance of a mock clock. +// The current time of the mock clock on initialization is the Unix epoch. +func NewMock() *Mock { + return &Mock{now: time.Unix(0, 0)} +} + +// Add moves the current time of the mock clock forward by the duration. +// This should only be called from a single goroutine at a time. +func (m *Mock) Add(d time.Duration) { + // Calculate the final current time. + t := m.now.Add(d) + + // Continue to execute timers until there are no more before the new time. + for { + if !m.runNextTimer(t) { + break + } + } + + // Ensure that we end with the new time. + m.mu.Lock() + m.now = t + m.mu.Unlock() + + // Give a small buffer to make sure the other goroutines get handled. + gosched() +} + +// runNextTimer executes the next timer in chronological order and moves the +// current time to the timer's next tick time. The next time is not executed if +// it's next time if after the max time. Returns true if a timer is executed. +func (m *Mock) runNextTimer(max time.Time) bool { + m.mu.Lock() + + // Sort timers by time. + sort.Sort(m.timers) + + // If we have no more timers then exit. + if len(m.timers) == 0 { + m.mu.Unlock() + return false + } + + // Retrieve next timer. Exit if next tick is after new time. + t := m.timers[0] + if t.Next().After(max) { + m.mu.Unlock() + return false + } + + // Move "now" forward and unlock clock. + m.now = t.Next() + m.mu.Unlock() + + // Execute timer. + t.Tick(m.now) + return true +} + +// After waits for the duration to elapse and then sends the current time on the returned channel. +func (m *Mock) After(d time.Duration) <-chan time.Time { + defer m.inc(&m.calls.After) + return m.Timer(d).C +} + +// AfterFunc waits for the duration to elapse and then executes a function. +// A Timer is returned that can be stopped. +func (m *Mock) AfterFunc(d time.Duration, f func()) *Timer { + defer m.inc(&m.calls.AfterFunc) + t := m.Timer(d) + t.C = nil + t.fn = f + return t +} + +// Now returns the current wall time on the mock clock. +func (m *Mock) Now() time.Time { + defer m.inc(&m.calls.Now) + m.mu.Lock() + defer m.mu.Unlock() + return m.now +} + +// Sleep pauses the goroutine for the given duration on the mock clock. +// The clock must be moved forward in a separate goroutine. +func (m *Mock) Sleep(d time.Duration) { + defer m.inc(&m.calls.Sleep) + <-m.After(d) +} + +// Tick is a convenience function for Ticker(). +// It will return a ticker channel that cannot be stopped. +func (m *Mock) Tick(d time.Duration) <-chan time.Time { + defer m.inc(&m.calls.Tick) + return m.Ticker(d).C +} + +// Ticker creates a new instance of Ticker. +func (m *Mock) Ticker(d time.Duration) *Ticker { + defer m.inc(&m.calls.Ticker) + m.mu.Lock() + defer m.mu.Unlock() + ch := make(chan time.Time) + t := &Ticker{ + C: ch, + c: ch, + mock: m, + d: d, + next: m.now.Add(d), + } + m.timers = append(m.timers, (*internalTicker)(t)) + return t +} + +// Timer creates a new instance of Timer. +func (m *Mock) Timer(d time.Duration) *Timer { + defer m.inc(&m.calls.Timer) + m.mu.Lock() + defer m.mu.Unlock() + ch := make(chan time.Time) + t := &Timer{ + C: ch, + c: ch, + mock: m, + next: m.now.Add(d), + } + m.timers = append(m.timers, (*internalTimer)(t)) + return t +} + +func (m *Mock) removeClockTimer(t clockTimer) { + m.mu.Lock() + defer m.mu.Unlock() + for i, timer := range m.timers { + if timer == t { + copy(m.timers[i:], m.timers[i+1:]) + m.timers[len(m.timers)-1] = nil + m.timers = m.timers[:len(m.timers)-1] + break + } + } + sort.Sort(m.timers) +} + +func (m *Mock) inc(addr *uint32) { + m.callsMutex.Lock() + defer m.callsMutex.Unlock() + *addr++ + var newWaiting []waiting + for _, w := range m.waiting { + if m.calls.atLeast(w.expected) { + close(w.done) + continue + } + newWaiting = append(newWaiting, w) + } + m.waiting = newWaiting +} + +// Wait waits for at least the relevant calls before returning. The expected +// Calls are always over the lifetime of the Mock. Values in the Calls struct +// are used as the minimum number of calls, this allows you to wait for only +// the calls you care about. +func (m *Mock) Wait(s Calls) { + m.callsMutex.Lock() + if m.calls.atLeast(s) { + m.callsMutex.Unlock() + return + } + done := make(chan struct{}) + m.waiting = append(m.waiting, waiting{expected: s, done: done}) + m.callsMutex.Unlock() + <-done +} + +// clockTimer represents an object with an associated start time. +type clockTimer interface { + Next() time.Time + Tick(time.Time) +} + +// clockTimers represents a list of sortable timers. +type clockTimers []clockTimer + +func (a clockTimers) Len() int { return len(a) } +func (a clockTimers) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a clockTimers) Less(i, j int) bool { return a[i].Next().Before(a[j].Next()) } + +// Timer represents a single event. +// The current time will be sent on C, unless the timer was created by AfterFunc. +type Timer struct { + C <-chan time.Time + c chan time.Time + timer *time.Timer // realtime impl, if set + next time.Time // next tick time + mock *Mock // mock clock, if set + fn func() // AfterFunc function, if set +} + +// Stop turns off the ticker. +func (t *Timer) Stop() { + if t.timer != nil { + t.timer.Stop() + } else { + t.mock.removeClockTimer((*internalTimer)(t)) + } +} + +type internalTimer Timer + +func (t *internalTimer) Next() time.Time { return t.next } +func (t *internalTimer) Tick(now time.Time) { + if t.fn != nil { + t.fn() + } else { + t.c <- now + } + t.mock.removeClockTimer((*internalTimer)(t)) + gosched() +} + +// Ticker holds a channel that receives "ticks" at regular intervals. +type Ticker struct { + C <-chan time.Time + c chan time.Time + ticker *time.Ticker // realtime impl, if set + next time.Time // next tick time + mock *Mock // mock clock, if set + d time.Duration // time between ticks +} + +// Stop turns off the ticker. +func (t *Ticker) Stop() { + if t.ticker != nil { + t.ticker.Stop() + } else { + t.mock.removeClockTimer((*internalTicker)(t)) + } +} + +type internalTicker Ticker + +func (t *internalTicker) Next() time.Time { return t.next } +func (t *internalTicker) Tick(now time.Time) { + select { + case t.c <- now: + case <-time.After(1 * time.Millisecond): + } + t.next = now.Add(t.d) + gosched() +} + +// Sleep momentarily so that other goroutines can process. +func gosched() { runtime.Gosched() } + +// Calls keeps track of the count of calls for each of the methods on the Clock +// interface. +type Calls struct { + After uint32 + AfterFunc uint32 + Now uint32 + Sleep uint32 + Tick uint32 + Ticker uint32 + Timer uint32 +} + +// atLeast returns true if at least the number of calls in o have been made. +func (c Calls) atLeast(o Calls) bool { + if c.After < o.After { + return false + } + if c.AfterFunc < o.AfterFunc { + return false + } + if c.Now < o.Now { + return false + } + if c.Sleep < o.Sleep { + return false + } + if c.Tick < o.Tick { + return false + } + if c.Ticker < o.Ticker { + return false + } + if c.Timer < o.Timer { + return false + } + return true +} + +type waiting struct { + expected Calls + done chan struct{} +} diff --git a/vendor/github.com/gogo/protobuf/jsonpb/jsonpb.go b/vendor/github.com/gogo/protobuf/jsonpb/jsonpb.go new file mode 100644 index 00000000000..e8134ec8ba3 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/jsonpb/jsonpb.go @@ -0,0 +1,1435 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2015 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +/* +Package jsonpb provides marshaling and unmarshaling between protocol buffers and JSON. +It follows the specification at https://developers.google.com/protocol-buffers/docs/proto3#json. + +This package produces a different output than the standard "encoding/json" package, +which does not operate correctly on protocol buffers. +*/ +package jsonpb + +import ( + "bytes" + "encoding/json" + "errors" + "fmt" + "io" + "math" + "reflect" + "sort" + "strconv" + "strings" + "time" + + "github.com/gogo/protobuf/proto" + "github.com/gogo/protobuf/types" +) + +const secondInNanos = int64(time.Second / time.Nanosecond) +const maxSecondsInDuration = 315576000000 + +// Marshaler is a configurable object for converting between +// protocol buffer objects and a JSON representation for them. +type Marshaler struct { + // Whether to render enum values as integers, as opposed to string values. + EnumsAsInts bool + + // Whether to render fields with zero values. + EmitDefaults bool + + // A string to indent each level by. The presence of this field will + // also cause a space to appear between the field separator and + // value, and for newlines to be appear between fields and array + // elements. + Indent string + + // Whether to use the original (.proto) name for fields. + OrigName bool + + // A custom URL resolver to use when marshaling Any messages to JSON. + // If unset, the default resolution strategy is to extract the + // fully-qualified type name from the type URL and pass that to + // proto.MessageType(string). + AnyResolver AnyResolver +} + +// AnyResolver takes a type URL, present in an Any message, and resolves it into +// an instance of the associated message. +type AnyResolver interface { + Resolve(typeUrl string) (proto.Message, error) +} + +func defaultResolveAny(typeUrl string) (proto.Message, error) { + // Only the part of typeUrl after the last slash is relevant. + mname := typeUrl + if slash := strings.LastIndex(mname, "/"); slash >= 0 { + mname = mname[slash+1:] + } + mt := proto.MessageType(mname) + if mt == nil { + return nil, fmt.Errorf("unknown message type %q", mname) + } + return reflect.New(mt.Elem()).Interface().(proto.Message), nil +} + +// JSONPBMarshaler is implemented by protobuf messages that customize the +// way they are marshaled to JSON. Messages that implement this should +// also implement JSONPBUnmarshaler so that the custom format can be +// parsed. +// +// The JSON marshaling must follow the proto to JSON specification: +// https://developers.google.com/protocol-buffers/docs/proto3#json +type JSONPBMarshaler interface { + MarshalJSONPB(*Marshaler) ([]byte, error) +} + +// JSONPBUnmarshaler is implemented by protobuf messages that customize +// the way they are unmarshaled from JSON. Messages that implement this +// should also implement JSONPBMarshaler so that the custom format can be +// produced. +// +// The JSON unmarshaling must follow the JSON to proto specification: +// https://developers.google.com/protocol-buffers/docs/proto3#json +type JSONPBUnmarshaler interface { + UnmarshalJSONPB(*Unmarshaler, []byte) error +} + +// Marshal marshals a protocol buffer into JSON. +func (m *Marshaler) Marshal(out io.Writer, pb proto.Message) error { + v := reflect.ValueOf(pb) + if pb == nil || (v.Kind() == reflect.Ptr && v.IsNil()) { + return errors.New("Marshal called with nil") + } + // Check for unset required fields first. + if err := checkRequiredFields(pb); err != nil { + return err + } + writer := &errWriter{writer: out} + return m.marshalObject(writer, pb, "", "") +} + +// MarshalToString converts a protocol buffer object to JSON string. +func (m *Marshaler) MarshalToString(pb proto.Message) (string, error) { + var buf bytes.Buffer + if err := m.Marshal(&buf, pb); err != nil { + return "", err + } + return buf.String(), nil +} + +type int32Slice []int32 + +var nonFinite = map[string]float64{ + `"NaN"`: math.NaN(), + `"Infinity"`: math.Inf(1), + `"-Infinity"`: math.Inf(-1), +} + +// For sorting extensions ids to ensure stable output. +func (s int32Slice) Len() int { return len(s) } +func (s int32Slice) Less(i, j int) bool { return s[i] < s[j] } +func (s int32Slice) Swap(i, j int) { s[i], s[j] = s[j], s[i] } + +type isWkt interface { + XXX_WellKnownType() string +} + +var ( + wktType = reflect.TypeOf((*isWkt)(nil)).Elem() + messageType = reflect.TypeOf((*proto.Message)(nil)).Elem() +) + +// marshalObject writes a struct to the Writer. +func (m *Marshaler) marshalObject(out *errWriter, v proto.Message, indent, typeURL string) error { + if jsm, ok := v.(JSONPBMarshaler); ok { + b, err := jsm.MarshalJSONPB(m) + if err != nil { + return err + } + if typeURL != "" { + // we are marshaling this object to an Any type + var js map[string]*json.RawMessage + if err = json.Unmarshal(b, &js); err != nil { + return fmt.Errorf("type %T produced invalid JSON: %v", v, err) + } + turl, err := json.Marshal(typeURL) + if err != nil { + return fmt.Errorf("failed to marshal type URL %q to JSON: %v", typeURL, err) + } + js["@type"] = (*json.RawMessage)(&turl) + if m.Indent != "" { + b, err = json.MarshalIndent(js, indent, m.Indent) + } else { + b, err = json.Marshal(js) + } + if err != nil { + return err + } + } + + out.write(string(b)) + return out.err + } + + s := reflect.ValueOf(v).Elem() + + // Handle well-known types. + if wkt, ok := v.(isWkt); ok { + switch wkt.XXX_WellKnownType() { + case "DoubleValue", "FloatValue", "Int64Value", "UInt64Value", + "Int32Value", "UInt32Value", "BoolValue", "StringValue", "BytesValue": + // "Wrappers use the same representation in JSON + // as the wrapped primitive type, ..." + sprop := proto.GetProperties(s.Type()) + return m.marshalValue(out, sprop.Prop[0], s.Field(0), indent) + case "Any": + // Any is a bit more involved. + return m.marshalAny(out, v, indent) + case "Duration": + s, ns := s.Field(0).Int(), s.Field(1).Int() + if s < -maxSecondsInDuration || s > maxSecondsInDuration { + return fmt.Errorf("seconds out of range %v", s) + } + if ns <= -secondInNanos || ns >= secondInNanos { + return fmt.Errorf("ns out of range (%v, %v)", -secondInNanos, secondInNanos) + } + if (s > 0 && ns < 0) || (s < 0 && ns > 0) { + return errors.New("signs of seconds and nanos do not match") + } + // Generated output always contains 0, 3, 6, or 9 fractional digits, + // depending on required precision, followed by the suffix "s". + f := "%d.%09d" + if ns < 0 { + ns = -ns + if s == 0 { + f = "-%d.%09d" + } + } + x := fmt.Sprintf(f, s, ns) + x = strings.TrimSuffix(x, "000") + x = strings.TrimSuffix(x, "000") + x = strings.TrimSuffix(x, ".000") + out.write(`"`) + out.write(x) + out.write(`s"`) + return out.err + case "Struct", "ListValue": + // Let marshalValue handle the `Struct.fields` map or the `ListValue.values` slice. + // TODO: pass the correct Properties if needed. + return m.marshalValue(out, &proto.Properties{}, s.Field(0), indent) + case "Timestamp": + // "RFC 3339, where generated output will always be Z-normalized + // and uses 0, 3, 6 or 9 fractional digits." + s, ns := s.Field(0).Int(), s.Field(1).Int() + if ns < 0 || ns >= secondInNanos { + return fmt.Errorf("ns out of range [0, %v)", secondInNanos) + } + t := time.Unix(s, ns).UTC() + // time.RFC3339Nano isn't exactly right (we need to get 3/6/9 fractional digits). + x := t.Format("2006-01-02T15:04:05.000000000") + x = strings.TrimSuffix(x, "000") + x = strings.TrimSuffix(x, "000") + x = strings.TrimSuffix(x, ".000") + out.write(`"`) + out.write(x) + out.write(`Z"`) + return out.err + case "Value": + // Value has a single oneof. + kind := s.Field(0) + if kind.IsNil() { + // "absence of any variant indicates an error" + return errors.New("nil Value") + } + // oneof -> *T -> T -> T.F + x := kind.Elem().Elem().Field(0) + // TODO: pass the correct Properties if needed. + return m.marshalValue(out, &proto.Properties{}, x, indent) + } + } + + out.write("{") + if m.Indent != "" { + out.write("\n") + } + + firstField := true + + if typeURL != "" { + if err := m.marshalTypeURL(out, indent, typeURL); err != nil { + return err + } + firstField = false + } + + for i := 0; i < s.NumField(); i++ { + value := s.Field(i) + valueField := s.Type().Field(i) + if strings.HasPrefix(valueField.Name, "XXX_") { + continue + } + + //this is not a protobuf field + if valueField.Tag.Get("protobuf") == "" && valueField.Tag.Get("protobuf_oneof") == "" { + continue + } + + // IsNil will panic on most value kinds. + switch value.Kind() { + case reflect.Chan, reflect.Func, reflect.Interface: + if value.IsNil() { + continue + } + } + + if !m.EmitDefaults { + switch value.Kind() { + case reflect.Bool: + if !value.Bool() { + continue + } + case reflect.Int32, reflect.Int64: + if value.Int() == 0 { + continue + } + case reflect.Uint32, reflect.Uint64: + if value.Uint() == 0 { + continue + } + case reflect.Float32, reflect.Float64: + if value.Float() == 0 { + continue + } + case reflect.String: + if value.Len() == 0 { + continue + } + case reflect.Map, reflect.Ptr, reflect.Slice: + if value.IsNil() { + continue + } + } + } + + // Oneof fields need special handling. + if valueField.Tag.Get("protobuf_oneof") != "" { + // value is an interface containing &T{real_value}. + sv := value.Elem().Elem() // interface -> *T -> T + value = sv.Field(0) + valueField = sv.Type().Field(0) + } + prop := jsonProperties(valueField, m.OrigName) + if !firstField { + m.writeSep(out) + } + // If the map value is a cast type, it may not implement proto.Message, therefore + // allow the struct tag to declare the underlying message type. Change the property + // of the child types, use CustomType as a passer. CastType currently property is + // not used in json encoding. + if value.Kind() == reflect.Map { + if tag := valueField.Tag.Get("protobuf"); tag != "" { + for _, v := range strings.Split(tag, ",") { + if !strings.HasPrefix(v, "castvaluetype=") { + continue + } + v = strings.TrimPrefix(v, "castvaluetype=") + prop.MapValProp.CustomType = v + break + } + } + } + if err := m.marshalField(out, prop, value, indent); err != nil { + return err + } + firstField = false + } + + // Handle proto2 extensions. + if ep, ok := v.(proto.Message); ok { + extensions := proto.RegisteredExtensions(v) + // Sort extensions for stable output. + ids := make([]int32, 0, len(extensions)) + for id, desc := range extensions { + if !proto.HasExtension(ep, desc) { + continue + } + ids = append(ids, id) + } + sort.Sort(int32Slice(ids)) + for _, id := range ids { + desc := extensions[id] + if desc == nil { + // unknown extension + continue + } + ext, extErr := proto.GetExtension(ep, desc) + if extErr != nil { + return extErr + } + value := reflect.ValueOf(ext) + var prop proto.Properties + prop.Parse(desc.Tag) + prop.JSONName = fmt.Sprintf("[%s]", desc.Name) + if !firstField { + m.writeSep(out) + } + if err := m.marshalField(out, &prop, value, indent); err != nil { + return err + } + firstField = false + } + + } + + if m.Indent != "" { + out.write("\n") + out.write(indent) + } + out.write("}") + return out.err +} + +func (m *Marshaler) writeSep(out *errWriter) { + if m.Indent != "" { + out.write(",\n") + } else { + out.write(",") + } +} + +func (m *Marshaler) marshalAny(out *errWriter, any proto.Message, indent string) error { + // "If the Any contains a value that has a special JSON mapping, + // it will be converted as follows: {"@type": xxx, "value": yyy}. + // Otherwise, the value will be converted into a JSON object, + // and the "@type" field will be inserted to indicate the actual data type." + v := reflect.ValueOf(any).Elem() + turl := v.Field(0).String() + val := v.Field(1).Bytes() + + var msg proto.Message + var err error + if m.AnyResolver != nil { + msg, err = m.AnyResolver.Resolve(turl) + } else { + msg, err = defaultResolveAny(turl) + } + if err != nil { + return err + } + + if err := proto.Unmarshal(val, msg); err != nil { + return err + } + + if _, ok := msg.(isWkt); ok { + out.write("{") + if m.Indent != "" { + out.write("\n") + } + if err := m.marshalTypeURL(out, indent, turl); err != nil { + return err + } + m.writeSep(out) + if m.Indent != "" { + out.write(indent) + out.write(m.Indent) + out.write(`"value": `) + } else { + out.write(`"value":`) + } + if err := m.marshalObject(out, msg, indent+m.Indent, ""); err != nil { + return err + } + if m.Indent != "" { + out.write("\n") + out.write(indent) + } + out.write("}") + return out.err + } + + return m.marshalObject(out, msg, indent, turl) +} + +func (m *Marshaler) marshalTypeURL(out *errWriter, indent, typeURL string) error { + if m.Indent != "" { + out.write(indent) + out.write(m.Indent) + } + out.write(`"@type":`) + if m.Indent != "" { + out.write(" ") + } + b, err := json.Marshal(typeURL) + if err != nil { + return err + } + out.write(string(b)) + return out.err +} + +// marshalField writes field description and value to the Writer. +func (m *Marshaler) marshalField(out *errWriter, prop *proto.Properties, v reflect.Value, indent string) error { + if m.Indent != "" { + out.write(indent) + out.write(m.Indent) + } + out.write(`"`) + out.write(prop.JSONName) + out.write(`":`) + if m.Indent != "" { + out.write(" ") + } + if err := m.marshalValue(out, prop, v, indent); err != nil { + return err + } + return nil +} + +// marshalValue writes the value to the Writer. +func (m *Marshaler) marshalValue(out *errWriter, prop *proto.Properties, v reflect.Value, indent string) error { + + v = reflect.Indirect(v) + + // Handle nil pointer + if v.Kind() == reflect.Invalid { + out.write("null") + return out.err + } + + // Handle repeated elements. + if v.Kind() == reflect.Slice && v.Type().Elem().Kind() != reflect.Uint8 { + out.write("[") + comma := "" + for i := 0; i < v.Len(); i++ { + sliceVal := v.Index(i) + out.write(comma) + if m.Indent != "" { + out.write("\n") + out.write(indent) + out.write(m.Indent) + out.write(m.Indent) + } + if err := m.marshalValue(out, prop, sliceVal, indent+m.Indent); err != nil { + return err + } + comma = "," + } + if m.Indent != "" { + out.write("\n") + out.write(indent) + out.write(m.Indent) + } + out.write("]") + return out.err + } + + // Handle well-known types. + // Most are handled up in marshalObject (because 99% are messages). + if v.Type().Implements(wktType) { + wkt := v.Interface().(isWkt) + switch wkt.XXX_WellKnownType() { + case "NullValue": + out.write("null") + return out.err + } + } + + if t, ok := v.Interface().(time.Time); ok { + ts, err := types.TimestampProto(t) + if err != nil { + return err + } + return m.marshalValue(out, prop, reflect.ValueOf(ts), indent) + } + + if d, ok := v.Interface().(time.Duration); ok { + dur := types.DurationProto(d) + return m.marshalValue(out, prop, reflect.ValueOf(dur), indent) + } + + // Handle enumerations. + if !m.EnumsAsInts && prop.Enum != "" { + // Unknown enum values will are stringified by the proto library as their + // value. Such values should _not_ be quoted or they will be interpreted + // as an enum string instead of their value. + enumStr := v.Interface().(fmt.Stringer).String() + var valStr string + if v.Kind() == reflect.Ptr { + valStr = strconv.Itoa(int(v.Elem().Int())) + } else { + valStr = strconv.Itoa(int(v.Int())) + } + + if m, ok := v.Interface().(interface { + MarshalJSON() ([]byte, error) + }); ok { + data, err := m.MarshalJSON() + if err != nil { + return err + } + enumStr = string(data) + enumStr, err = strconv.Unquote(enumStr) + if err != nil { + return err + } + } + + isKnownEnum := enumStr != valStr + + if isKnownEnum { + out.write(`"`) + } + out.write(enumStr) + if isKnownEnum { + out.write(`"`) + } + return out.err + } + + // Handle nested messages. + if v.Kind() == reflect.Struct { + i := v + if v.CanAddr() { + i = v.Addr() + } else { + i = reflect.New(v.Type()) + i.Elem().Set(v) + } + iface := i.Interface() + if iface == nil { + out.write(`null`) + return out.err + } + + if m, ok := v.Interface().(interface { + MarshalJSON() ([]byte, error) + }); ok { + data, err := m.MarshalJSON() + if err != nil { + return err + } + out.write(string(data)) + return nil + } + + pm, ok := iface.(proto.Message) + if !ok { + if prop.CustomType == "" { + return fmt.Errorf("%v does not implement proto.Message", v.Type()) + } + t := proto.MessageType(prop.CustomType) + if t == nil || !i.Type().ConvertibleTo(t) { + return fmt.Errorf("%v declared custom type %s but it is not convertible to %v", v.Type(), prop.CustomType, t) + } + pm = i.Convert(t).Interface().(proto.Message) + } + return m.marshalObject(out, pm, indent+m.Indent, "") + } + + // Handle maps. + // Since Go randomizes map iteration, we sort keys for stable output. + if v.Kind() == reflect.Map { + out.write(`{`) + keys := v.MapKeys() + sort.Sort(mapKeys(keys)) + for i, k := range keys { + if i > 0 { + out.write(`,`) + } + if m.Indent != "" { + out.write("\n") + out.write(indent) + out.write(m.Indent) + out.write(m.Indent) + } + + // TODO handle map key prop properly + b, err := json.Marshal(k.Interface()) + if err != nil { + return err + } + s := string(b) + + // If the JSON is not a string value, encode it again to make it one. + if !strings.HasPrefix(s, `"`) { + b, err := json.Marshal(s) + if err != nil { + return err + } + s = string(b) + } + + out.write(s) + out.write(`:`) + if m.Indent != "" { + out.write(` `) + } + + vprop := prop + if prop != nil && prop.MapValProp != nil { + vprop = prop.MapValProp + } + if err := m.marshalValue(out, vprop, v.MapIndex(k), indent+m.Indent); err != nil { + return err + } + } + if m.Indent != "" { + out.write("\n") + out.write(indent) + out.write(m.Indent) + } + out.write(`}`) + return out.err + } + + // Handle non-finite floats, e.g. NaN, Infinity and -Infinity. + if v.Kind() == reflect.Float32 || v.Kind() == reflect.Float64 { + f := v.Float() + var sval string + switch { + case math.IsInf(f, 1): + sval = `"Infinity"` + case math.IsInf(f, -1): + sval = `"-Infinity"` + case math.IsNaN(f): + sval = `"NaN"` + } + if sval != "" { + out.write(sval) + return out.err + } + } + + // Default handling defers to the encoding/json library. + b, err := json.Marshal(v.Interface()) + if err != nil { + return err + } + needToQuote := string(b[0]) != `"` && (v.Kind() == reflect.Int64 || v.Kind() == reflect.Uint64) + if needToQuote { + out.write(`"`) + } + out.write(string(b)) + if needToQuote { + out.write(`"`) + } + return out.err +} + +// Unmarshaler is a configurable object for converting from a JSON +// representation to a protocol buffer object. +type Unmarshaler struct { + // Whether to allow messages to contain unknown fields, as opposed to + // failing to unmarshal. + AllowUnknownFields bool + + // A custom URL resolver to use when unmarshaling Any messages from JSON. + // If unset, the default resolution strategy is to extract the + // fully-qualified type name from the type URL and pass that to + // proto.MessageType(string). + AnyResolver AnyResolver +} + +// UnmarshalNext unmarshals the next protocol buffer from a JSON object stream. +// This function is lenient and will decode any options permutations of the +// related Marshaler. +func (u *Unmarshaler) UnmarshalNext(dec *json.Decoder, pb proto.Message) error { + inputValue := json.RawMessage{} + if err := dec.Decode(&inputValue); err != nil { + return err + } + if err := u.unmarshalValue(reflect.ValueOf(pb).Elem(), inputValue, nil); err != nil { + return err + } + return checkRequiredFields(pb) +} + +// Unmarshal unmarshals a JSON object stream into a protocol +// buffer. This function is lenient and will decode any options +// permutations of the related Marshaler. +func (u *Unmarshaler) Unmarshal(r io.Reader, pb proto.Message) error { + dec := json.NewDecoder(r) + return u.UnmarshalNext(dec, pb) +} + +// UnmarshalNext unmarshals the next protocol buffer from a JSON object stream. +// This function is lenient and will decode any options permutations of the +// related Marshaler. +func UnmarshalNext(dec *json.Decoder, pb proto.Message) error { + return new(Unmarshaler).UnmarshalNext(dec, pb) +} + +// Unmarshal unmarshals a JSON object stream into a protocol +// buffer. This function is lenient and will decode any options +// permutations of the related Marshaler. +func Unmarshal(r io.Reader, pb proto.Message) error { + return new(Unmarshaler).Unmarshal(r, pb) +} + +// UnmarshalString will populate the fields of a protocol buffer based +// on a JSON string. This function is lenient and will decode any options +// permutations of the related Marshaler. +func UnmarshalString(str string, pb proto.Message) error { + return new(Unmarshaler).Unmarshal(strings.NewReader(str), pb) +} + +// unmarshalValue converts/copies a value into the target. +// prop may be nil. +func (u *Unmarshaler) unmarshalValue(target reflect.Value, inputValue json.RawMessage, prop *proto.Properties) error { + targetType := target.Type() + + // Allocate memory for pointer fields. + if targetType.Kind() == reflect.Ptr { + // If input value is "null" and target is a pointer type, then the field should be treated as not set + // UNLESS the target is structpb.Value, in which case it should be set to structpb.NullValue. + _, isJSONPBUnmarshaler := target.Interface().(JSONPBUnmarshaler) + if string(inputValue) == "null" && targetType != reflect.TypeOf(&types.Value{}) && !isJSONPBUnmarshaler { + return nil + } + target.Set(reflect.New(targetType.Elem())) + + return u.unmarshalValue(target.Elem(), inputValue, prop) + } + + if jsu, ok := target.Addr().Interface().(JSONPBUnmarshaler); ok { + return jsu.UnmarshalJSONPB(u, []byte(inputValue)) + } + + // Handle well-known types that are not pointers. + if w, ok := target.Addr().Interface().(isWkt); ok { + switch w.XXX_WellKnownType() { + case "DoubleValue", "FloatValue", "Int64Value", "UInt64Value", + "Int32Value", "UInt32Value", "BoolValue", "StringValue", "BytesValue": + return u.unmarshalValue(target.Field(0), inputValue, prop) + case "Any": + // Use json.RawMessage pointer type instead of value to support pre-1.8 version. + // 1.8 changed RawMessage.MarshalJSON from pointer type to value type, see + // https://github.com/golang/go/issues/14493 + var jsonFields map[string]*json.RawMessage + if err := json.Unmarshal(inputValue, &jsonFields); err != nil { + return err + } + + val, ok := jsonFields["@type"] + if !ok || val == nil { + return errors.New("Any JSON doesn't have '@type'") + } + + var turl string + if err := json.Unmarshal([]byte(*val), &turl); err != nil { + return fmt.Errorf("can't unmarshal Any's '@type': %q", *val) + } + target.Field(0).SetString(turl) + + var m proto.Message + var err error + if u.AnyResolver != nil { + m, err = u.AnyResolver.Resolve(turl) + } else { + m, err = defaultResolveAny(turl) + } + if err != nil { + return err + } + + if _, ok := m.(isWkt); ok { + val, ok := jsonFields["value"] + if !ok { + return errors.New("Any JSON doesn't have 'value'") + } + + if err = u.unmarshalValue(reflect.ValueOf(m).Elem(), *val, nil); err != nil { + return fmt.Errorf("can't unmarshal Any nested proto %T: %v", m, err) + } + } else { + delete(jsonFields, "@type") + nestedProto, uerr := json.Marshal(jsonFields) + if uerr != nil { + return fmt.Errorf("can't generate JSON for Any's nested proto to be unmarshaled: %v", uerr) + } + + if err = u.unmarshalValue(reflect.ValueOf(m).Elem(), nestedProto, nil); err != nil { + return fmt.Errorf("can't unmarshal Any nested proto %T: %v", m, err) + } + } + + b, err := proto.Marshal(m) + if err != nil { + return fmt.Errorf("can't marshal proto %T into Any.Value: %v", m, err) + } + target.Field(1).SetBytes(b) + + return nil + case "Duration": + unq, err := unquote(string(inputValue)) + if err != nil { + return err + } + + d, err := time.ParseDuration(unq) + if err != nil { + return fmt.Errorf("bad Duration: %v", err) + } + + ns := d.Nanoseconds() + s := ns / 1e9 + ns %= 1e9 + target.Field(0).SetInt(s) + target.Field(1).SetInt(ns) + return nil + case "Timestamp": + unq, err := unquote(string(inputValue)) + if err != nil { + return err + } + + t, err := time.Parse(time.RFC3339Nano, unq) + if err != nil { + return fmt.Errorf("bad Timestamp: %v", err) + } + + target.Field(0).SetInt(t.Unix()) + target.Field(1).SetInt(int64(t.Nanosecond())) + return nil + case "Struct": + var m map[string]json.RawMessage + if err := json.Unmarshal(inputValue, &m); err != nil { + return fmt.Errorf("bad StructValue: %v", err) + } + target.Field(0).Set(reflect.ValueOf(map[string]*types.Value{})) + for k, jv := range m { + pv := &types.Value{} + if err := u.unmarshalValue(reflect.ValueOf(pv).Elem(), jv, prop); err != nil { + return fmt.Errorf("bad value in StructValue for key %q: %v", k, err) + } + target.Field(0).SetMapIndex(reflect.ValueOf(k), reflect.ValueOf(pv)) + } + return nil + case "ListValue": + var s []json.RawMessage + if err := json.Unmarshal(inputValue, &s); err != nil { + return fmt.Errorf("bad ListValue: %v", err) + } + + target.Field(0).Set(reflect.ValueOf(make([]*types.Value, len(s)))) + for i, sv := range s { + if err := u.unmarshalValue(target.Field(0).Index(i), sv, prop); err != nil { + return err + } + } + return nil + case "Value": + ivStr := string(inputValue) + if ivStr == "null" { + target.Field(0).Set(reflect.ValueOf(&types.Value_NullValue{})) + } else if v, err := strconv.ParseFloat(ivStr, 0); err == nil { + target.Field(0).Set(reflect.ValueOf(&types.Value_NumberValue{NumberValue: v})) + } else if v, err := unquote(ivStr); err == nil { + target.Field(0).Set(reflect.ValueOf(&types.Value_StringValue{StringValue: v})) + } else if v, err := strconv.ParseBool(ivStr); err == nil { + target.Field(0).Set(reflect.ValueOf(&types.Value_BoolValue{BoolValue: v})) + } else if err := json.Unmarshal(inputValue, &[]json.RawMessage{}); err == nil { + lv := &types.ListValue{} + target.Field(0).Set(reflect.ValueOf(&types.Value_ListValue{ListValue: lv})) + return u.unmarshalValue(reflect.ValueOf(lv).Elem(), inputValue, prop) + } else if err := json.Unmarshal(inputValue, &map[string]json.RawMessage{}); err == nil { + sv := &types.Struct{} + target.Field(0).Set(reflect.ValueOf(&types.Value_StructValue{StructValue: sv})) + return u.unmarshalValue(reflect.ValueOf(sv).Elem(), inputValue, prop) + } else { + return fmt.Errorf("unrecognized type for Value %q", ivStr) + } + return nil + } + } + + if t, ok := target.Addr().Interface().(*time.Time); ok { + ts := &types.Timestamp{} + if err := u.unmarshalValue(reflect.ValueOf(ts).Elem(), inputValue, prop); err != nil { + return err + } + tt, err := types.TimestampFromProto(ts) + if err != nil { + return err + } + *t = tt + return nil + } + + if d, ok := target.Addr().Interface().(*time.Duration); ok { + dur := &types.Duration{} + if err := u.unmarshalValue(reflect.ValueOf(dur).Elem(), inputValue, prop); err != nil { + return err + } + dd, err := types.DurationFromProto(dur) + if err != nil { + return err + } + *d = dd + return nil + } + + // Handle enums, which have an underlying type of int32, + // and may appear as strings. + // The case of an enum appearing as a number is handled + // at the bottom of this function. + if inputValue[0] == '"' && prop != nil && prop.Enum != "" { + vmap := proto.EnumValueMap(prop.Enum) + // Don't need to do unquoting; valid enum names + // are from a limited character set. + s := inputValue[1 : len(inputValue)-1] + n, ok := vmap[string(s)] + if !ok { + return fmt.Errorf("unknown value %q for enum %s", s, prop.Enum) + } + if target.Kind() == reflect.Ptr { // proto2 + target.Set(reflect.New(targetType.Elem())) + target = target.Elem() + } + if targetType.Kind() != reflect.Int32 { + return fmt.Errorf("invalid target %q for enum %s", targetType.Kind(), prop.Enum) + } + target.SetInt(int64(n)) + return nil + } + + if prop != nil && len(prop.CustomType) > 0 && target.CanAddr() { + if m, ok := target.Addr().Interface().(interface { + UnmarshalJSON([]byte) error + }); ok { + return json.Unmarshal(inputValue, m) + } + } + + // Handle nested messages. + if targetType.Kind() == reflect.Struct { + var jsonFields map[string]json.RawMessage + if err := json.Unmarshal(inputValue, &jsonFields); err != nil { + return err + } + + consumeField := func(prop *proto.Properties) (json.RawMessage, bool) { + // Be liberal in what names we accept; both orig_name and camelName are okay. + fieldNames := acceptedJSONFieldNames(prop) + + vOrig, okOrig := jsonFields[fieldNames.orig] + vCamel, okCamel := jsonFields[fieldNames.camel] + if !okOrig && !okCamel { + return nil, false + } + // If, for some reason, both are present in the data, favour the camelName. + var raw json.RawMessage + if okOrig { + raw = vOrig + delete(jsonFields, fieldNames.orig) + } + if okCamel { + raw = vCamel + delete(jsonFields, fieldNames.camel) + } + return raw, true + } + + sprops := proto.GetProperties(targetType) + for i := 0; i < target.NumField(); i++ { + ft := target.Type().Field(i) + if strings.HasPrefix(ft.Name, "XXX_") { + continue + } + valueForField, ok := consumeField(sprops.Prop[i]) + if !ok { + continue + } + + if err := u.unmarshalValue(target.Field(i), valueForField, sprops.Prop[i]); err != nil { + return err + } + } + // Check for any oneof fields. + if len(jsonFields) > 0 { + for _, oop := range sprops.OneofTypes { + raw, ok := consumeField(oop.Prop) + if !ok { + continue + } + nv := reflect.New(oop.Type.Elem()) + target.Field(oop.Field).Set(nv) + if err := u.unmarshalValue(nv.Elem().Field(0), raw, oop.Prop); err != nil { + return err + } + } + } + // Handle proto2 extensions. + if len(jsonFields) > 0 { + if ep, ok := target.Addr().Interface().(proto.Message); ok { + for _, ext := range proto.RegisteredExtensions(ep) { + name := fmt.Sprintf("[%s]", ext.Name) + raw, ok := jsonFields[name] + if !ok { + continue + } + delete(jsonFields, name) + nv := reflect.New(reflect.TypeOf(ext.ExtensionType).Elem()) + if err := u.unmarshalValue(nv.Elem(), raw, nil); err != nil { + return err + } + if err := proto.SetExtension(ep, ext, nv.Interface()); err != nil { + return err + } + } + } + } + if !u.AllowUnknownFields && len(jsonFields) > 0 { + // Pick any field to be the scapegoat. + var f string + for fname := range jsonFields { + f = fname + break + } + return fmt.Errorf("unknown field %q in %v", f, targetType) + } + return nil + } + + // Handle arrays + if targetType.Kind() == reflect.Slice { + if targetType.Elem().Kind() == reflect.Uint8 { + outRef := reflect.New(targetType) + outVal := outRef.Interface() + //CustomType with underlying type []byte + if _, ok := outVal.(interface { + UnmarshalJSON([]byte) error + }); ok { + if err := json.Unmarshal(inputValue, outVal); err != nil { + return err + } + target.Set(outRef.Elem()) + return nil + } + // Special case for encoded bytes. Pre-go1.5 doesn't support unmarshalling + // strings into aliased []byte types. + // https://github.com/golang/go/commit/4302fd0409da5e4f1d71471a6770dacdc3301197 + // https://github.com/golang/go/commit/c60707b14d6be26bf4213114d13070bff00d0b0a + var out []byte + if err := json.Unmarshal(inputValue, &out); err != nil { + return err + } + target.SetBytes(out) + return nil + } + + var slc []json.RawMessage + if err := json.Unmarshal(inputValue, &slc); err != nil { + return err + } + if slc != nil { + l := len(slc) + target.Set(reflect.MakeSlice(targetType, l, l)) + for i := 0; i < l; i++ { + if err := u.unmarshalValue(target.Index(i), slc[i], prop); err != nil { + return err + } + } + } + return nil + } + + // Handle maps (whose keys are always strings) + if targetType.Kind() == reflect.Map { + var mp map[string]json.RawMessage + if err := json.Unmarshal(inputValue, &mp); err != nil { + return err + } + if mp != nil { + target.Set(reflect.MakeMap(targetType)) + for ks, raw := range mp { + // Unmarshal map key. The core json library already decoded the key into a + // string, so we handle that specially. Other types were quoted post-serialization. + var k reflect.Value + if targetType.Key().Kind() == reflect.String { + k = reflect.ValueOf(ks) + } else { + k = reflect.New(targetType.Key()).Elem() + var kprop *proto.Properties + if prop != nil && prop.MapKeyProp != nil { + kprop = prop.MapKeyProp + } + if err := u.unmarshalValue(k, json.RawMessage(ks), kprop); err != nil { + return err + } + } + + if !k.Type().AssignableTo(targetType.Key()) { + k = k.Convert(targetType.Key()) + } + + // Unmarshal map value. + v := reflect.New(targetType.Elem()).Elem() + var vprop *proto.Properties + if prop != nil && prop.MapValProp != nil { + vprop = prop.MapValProp + } + if err := u.unmarshalValue(v, raw, vprop); err != nil { + return err + } + target.SetMapIndex(k, v) + } + } + return nil + } + + // Non-finite numbers can be encoded as strings. + isFloat := targetType.Kind() == reflect.Float32 || targetType.Kind() == reflect.Float64 + if isFloat { + if num, ok := nonFinite[string(inputValue)]; ok { + target.SetFloat(num) + return nil + } + } + + // integers & floats can be encoded as strings. In this case we drop + // the quotes and proceed as normal. + isNum := targetType.Kind() == reflect.Int64 || targetType.Kind() == reflect.Uint64 || + targetType.Kind() == reflect.Int32 || targetType.Kind() == reflect.Uint32 || + targetType.Kind() == reflect.Float32 || targetType.Kind() == reflect.Float64 + if isNum && strings.HasPrefix(string(inputValue), `"`) { + inputValue = inputValue[1 : len(inputValue)-1] + } + + // Use the encoding/json for parsing other value types. + return json.Unmarshal(inputValue, target.Addr().Interface()) +} + +func unquote(s string) (string, error) { + var ret string + err := json.Unmarshal([]byte(s), &ret) + return ret, err +} + +// jsonProperties returns parsed proto.Properties for the field and corrects JSONName attribute. +func jsonProperties(f reflect.StructField, origName bool) *proto.Properties { + var prop proto.Properties + prop.Init(f.Type, f.Name, f.Tag.Get("protobuf"), &f) + if origName || prop.JSONName == "" { + prop.JSONName = prop.OrigName + } + return &prop +} + +type fieldNames struct { + orig, camel string +} + +func acceptedJSONFieldNames(prop *proto.Properties) fieldNames { + opts := fieldNames{orig: prop.OrigName, camel: prop.OrigName} + if prop.JSONName != "" { + opts.camel = prop.JSONName + } + return opts +} + +// Writer wrapper inspired by https://blog.golang.org/errors-are-values +type errWriter struct { + writer io.Writer + err error +} + +func (w *errWriter) write(str string) { + if w.err != nil { + return + } + _, w.err = w.writer.Write([]byte(str)) +} + +// Map fields may have key types of non-float scalars, strings and enums. +// The easiest way to sort them in some deterministic order is to use fmt. +// If this turns out to be inefficient we can always consider other options, +// such as doing a Schwartzian transform. +// +// Numeric keys are sorted in numeric order per +// https://developers.google.com/protocol-buffers/docs/proto#maps. +type mapKeys []reflect.Value + +func (s mapKeys) Len() int { return len(s) } +func (s mapKeys) Swap(i, j int) { s[i], s[j] = s[j], s[i] } +func (s mapKeys) Less(i, j int) bool { + if k := s[i].Kind(); k == s[j].Kind() { + switch k { + case reflect.String: + return s[i].String() < s[j].String() + case reflect.Int32, reflect.Int64: + return s[i].Int() < s[j].Int() + case reflect.Uint32, reflect.Uint64: + return s[i].Uint() < s[j].Uint() + } + } + return fmt.Sprint(s[i].Interface()) < fmt.Sprint(s[j].Interface()) +} + +// checkRequiredFields returns an error if any required field in the given proto message is not set. +// This function is used by both Marshal and Unmarshal. While required fields only exist in a +// proto2 message, a proto3 message can contain proto2 message(s). +func checkRequiredFields(pb proto.Message) error { + // Most well-known type messages do not contain required fields. The "Any" type may contain + // a message that has required fields. + // + // When an Any message is being marshaled, the code will invoked proto.Unmarshal on Any.Value + // field in order to transform that into JSON, and that should have returned an error if a + // required field is not set in the embedded message. + // + // When an Any message is being unmarshaled, the code will have invoked proto.Marshal on the + // embedded message to store the serialized message in Any.Value field, and that should have + // returned an error if a required field is not set. + if _, ok := pb.(isWkt); ok { + return nil + } + + v := reflect.ValueOf(pb) + // Skip message if it is not a struct pointer. + if v.Kind() != reflect.Ptr { + return nil + } + v = v.Elem() + if v.Kind() != reflect.Struct { + return nil + } + + for i := 0; i < v.NumField(); i++ { + field := v.Field(i) + sfield := v.Type().Field(i) + + if sfield.PkgPath != "" { + // blank PkgPath means the field is exported; skip if not exported + continue + } + + if strings.HasPrefix(sfield.Name, "XXX_") { + continue + } + + // Oneof field is an interface implemented by wrapper structs containing the actual oneof + // field, i.e. an interface containing &T{real_value}. + if sfield.Tag.Get("protobuf_oneof") != "" { + if field.Kind() != reflect.Interface { + continue + } + v := field.Elem() + if v.Kind() != reflect.Ptr || v.IsNil() { + continue + } + v = v.Elem() + if v.Kind() != reflect.Struct || v.NumField() < 1 { + continue + } + field = v.Field(0) + sfield = v.Type().Field(0) + } + + protoTag := sfield.Tag.Get("protobuf") + if protoTag == "" { + continue + } + var prop proto.Properties + prop.Init(sfield.Type, sfield.Name, protoTag, &sfield) + + switch field.Kind() { + case reflect.Map: + if field.IsNil() { + continue + } + // Check each map value. + keys := field.MapKeys() + for _, k := range keys { + v := field.MapIndex(k) + if err := checkRequiredFieldsInValue(v); err != nil { + return err + } + } + case reflect.Slice: + // Handle non-repeated type, e.g. bytes. + if !prop.Repeated { + if prop.Required && field.IsNil() { + return fmt.Errorf("required field %q is not set", prop.Name) + } + continue + } + + // Handle repeated type. + if field.IsNil() { + continue + } + // Check each slice item. + for i := 0; i < field.Len(); i++ { + v := field.Index(i) + if err := checkRequiredFieldsInValue(v); err != nil { + return err + } + } + case reflect.Ptr: + if field.IsNil() { + if prop.Required { + return fmt.Errorf("required field %q is not set", prop.Name) + } + continue + } + if err := checkRequiredFieldsInValue(field); err != nil { + return err + } + } + } + + // Handle proto2 extensions. + for _, ext := range proto.RegisteredExtensions(pb) { + if !proto.HasExtension(pb, ext) { + continue + } + ep, err := proto.GetExtension(pb, ext) + if err != nil { + return err + } + err = checkRequiredFieldsInValue(reflect.ValueOf(ep)) + if err != nil { + return err + } + } + + return nil +} + +func checkRequiredFieldsInValue(v reflect.Value) error { + if v.Type().Implements(messageType) { + return checkRequiredFields(v.Interface().(proto.Message)) + } + return nil +} diff --git a/vendor/github.com/golang/mock/AUTHORS b/vendor/github.com/golang/mock/AUTHORS new file mode 100644 index 00000000000..660b8ccc8ae --- /dev/null +++ b/vendor/github.com/golang/mock/AUTHORS @@ -0,0 +1,12 @@ +# This is the official list of GoMock authors for copyright purposes. +# This file is distinct from the CONTRIBUTORS files. +# See the latter for an explanation. + +# Names should be added to this file as +# Name or Organization +# The email address is not required for organizations. + +# Please keep the list sorted. + +Alex Reece +Google Inc. diff --git a/vendor/github.com/golang/mock/CONTRIBUTORS b/vendor/github.com/golang/mock/CONTRIBUTORS new file mode 100644 index 00000000000..def849cab1b --- /dev/null +++ b/vendor/github.com/golang/mock/CONTRIBUTORS @@ -0,0 +1,37 @@ +# This is the official list of people who can contribute (and typically +# have contributed) code to the gomock repository. +# The AUTHORS file lists the copyright holders; this file +# lists people. For example, Google employees are listed here +# but not in AUTHORS, because Google holds the copyright. +# +# The submission process automatically checks to make sure +# that people submitting code are listed in this file (by email address). +# +# Names should be added to this file only after verifying that +# the individual or the individual's organization has agreed to +# the appropriate Contributor License Agreement, found here: +# +# http://code.google.com/legal/individual-cla-v1.0.html +# http://code.google.com/legal/corporate-cla-v1.0.html +# +# The agreement for individuals can be filled out on the web. +# +# When adding J Random Contributor's name to this file, +# either J's name or J's organization's name should be +# added to the AUTHORS file, depending on whether the +# individual or corporate CLA was used. + +# Names should be added to this file like so: +# Name +# +# An entry with two email addresses specifies that the +# first address should be used in the submit logs and +# that the second address should be recognized as the +# same person when interacting with Rietveld. + +# Please keep the list sorted. + +Aaron Jacobs +Alex Reece +David Symonds +Ryan Barrett diff --git a/vendor/github.com/golang/mock/LICENSE b/vendor/github.com/golang/mock/LICENSE new file mode 100644 index 00000000000..d6456956733 --- /dev/null +++ b/vendor/github.com/golang/mock/LICENSE @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/vendor/github.com/golang/mock/gomock/call.go b/vendor/github.com/golang/mock/gomock/call.go new file mode 100644 index 00000000000..13c9f44b1ef --- /dev/null +++ b/vendor/github.com/golang/mock/gomock/call.go @@ -0,0 +1,445 @@ +// Copyright 2010 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package gomock + +import ( + "fmt" + "reflect" + "strconv" + "strings" +) + +// Call represents an expected call to a mock. +type Call struct { + t TestHelper // for triggering test failures on invalid call setup + + receiver interface{} // the receiver of the method call + method string // the name of the method + methodType reflect.Type // the type of the method + args []Matcher // the args + origin string // file and line number of call setup + + preReqs []*Call // prerequisite calls + + // Expectations + minCalls, maxCalls int + + numCalls int // actual number made + + // actions are called when this Call is called. Each action gets the args and + // can set the return values by returning a non-nil slice. Actions run in the + // order they are created. + actions []func([]interface{}) []interface{} +} + +// newCall creates a *Call. It requires the method type in order to support +// unexported methods. +func newCall(t TestHelper, receiver interface{}, method string, methodType reflect.Type, args ...interface{}) *Call { + t.Helper() + + // TODO: check arity, types. + mArgs := make([]Matcher, len(args)) + for i, arg := range args { + if m, ok := arg.(Matcher); ok { + mArgs[i] = m + } else if arg == nil { + // Handle nil specially so that passing a nil interface value + // will match the typed nils of concrete args. + mArgs[i] = Nil() + } else { + mArgs[i] = Eq(arg) + } + } + + // callerInfo's skip should be updated if the number of calls between the user's test + // and this line changes, i.e. this code is wrapped in another anonymous function. + // 0 is us, 1 is RecordCallWithMethodType(), 2 is the generated recorder, and 3 is the user's test. + origin := callerInfo(3) + actions := []func([]interface{}) []interface{}{func([]interface{}) []interface{} { + // Synthesize the zero value for each of the return args' types. + rets := make([]interface{}, methodType.NumOut()) + for i := 0; i < methodType.NumOut(); i++ { + rets[i] = reflect.Zero(methodType.Out(i)).Interface() + } + return rets + }} + return &Call{t: t, receiver: receiver, method: method, methodType: methodType, + args: mArgs, origin: origin, minCalls: 1, maxCalls: 1, actions: actions} +} + +// AnyTimes allows the expectation to be called 0 or more times +func (c *Call) AnyTimes() *Call { + c.minCalls, c.maxCalls = 0, 1e8 // close enough to infinity + return c +} + +// MinTimes requires the call to occur at least n times. If AnyTimes or MaxTimes have not been called or if MaxTimes +// was previously called with 1, MinTimes also sets the maximum number of calls to infinity. +func (c *Call) MinTimes(n int) *Call { + c.minCalls = n + if c.maxCalls == 1 { + c.maxCalls = 1e8 + } + return c +} + +// MaxTimes limits the number of calls to n times. If AnyTimes or MinTimes have not been called or if MinTimes was +// previously called with 1, MaxTimes also sets the minimum number of calls to 0. +func (c *Call) MaxTimes(n int) *Call { + c.maxCalls = n + if c.minCalls == 1 { + c.minCalls = 0 + } + return c +} + +// DoAndReturn declares the action to run when the call is matched. +// The return values from this function are returned by the mocked function. +// It takes an interface{} argument to support n-arity functions. +func (c *Call) DoAndReturn(f interface{}) *Call { + // TODO: Check arity and types here, rather than dying badly elsewhere. + v := reflect.ValueOf(f) + + c.addAction(func(args []interface{}) []interface{} { + c.t.Helper() + vArgs := make([]reflect.Value, len(args)) + ft := v.Type() + if c.methodType.NumIn() != ft.NumIn() { + c.t.Fatalf("wrong number of arguments in DoAndReturn func for %T.%v: got %d, want %d [%s]", + c.receiver, c.method, ft.NumIn(), c.methodType.NumIn(), c.origin) + return nil + } + for i := 0; i < len(args); i++ { + if args[i] != nil { + vArgs[i] = reflect.ValueOf(args[i]) + } else { + // Use the zero value for the arg. + vArgs[i] = reflect.Zero(ft.In(i)) + } + } + vRets := v.Call(vArgs) + rets := make([]interface{}, len(vRets)) + for i, ret := range vRets { + rets[i] = ret.Interface() + } + return rets + }) + return c +} + +// Do declares the action to run when the call is matched. The function's +// return values are ignored to retain backward compatibility. To use the +// return values call DoAndReturn. +// It takes an interface{} argument to support n-arity functions. +func (c *Call) Do(f interface{}) *Call { + // TODO: Check arity and types here, rather than dying badly elsewhere. + v := reflect.ValueOf(f) + + c.addAction(func(args []interface{}) []interface{} { + c.t.Helper() + if c.methodType.NumIn() != v.Type().NumIn() { + c.t.Fatalf("wrong number of arguments in Do func for %T.%v: got %d, want %d [%s]", + c.receiver, c.method, v.Type().NumIn(), c.methodType.NumIn(), c.origin) + return nil + } + vArgs := make([]reflect.Value, len(args)) + ft := v.Type() + for i := 0; i < len(args); i++ { + if args[i] != nil { + vArgs[i] = reflect.ValueOf(args[i]) + } else { + // Use the zero value for the arg. + vArgs[i] = reflect.Zero(ft.In(i)) + } + } + v.Call(vArgs) + return nil + }) + return c +} + +// Return declares the values to be returned by the mocked function call. +func (c *Call) Return(rets ...interface{}) *Call { + c.t.Helper() + + mt := c.methodType + if len(rets) != mt.NumOut() { + c.t.Fatalf("wrong number of arguments to Return for %T.%v: got %d, want %d [%s]", + c.receiver, c.method, len(rets), mt.NumOut(), c.origin) + } + for i, ret := range rets { + if got, want := reflect.TypeOf(ret), mt.Out(i); got == want { + // Identical types; nothing to do. + } else if got == nil { + // Nil needs special handling. + switch want.Kind() { + case reflect.Chan, reflect.Func, reflect.Interface, reflect.Map, reflect.Ptr, reflect.Slice: + // ok + default: + c.t.Fatalf("argument %d to Return for %T.%v is nil, but %v is not nillable [%s]", + i, c.receiver, c.method, want, c.origin) + } + } else if got.AssignableTo(want) { + // Assignable type relation. Make the assignment now so that the generated code + // can return the values with a type assertion. + v := reflect.New(want).Elem() + v.Set(reflect.ValueOf(ret)) + rets[i] = v.Interface() + } else { + c.t.Fatalf("wrong type of argument %d to Return for %T.%v: %v is not assignable to %v [%s]", + i, c.receiver, c.method, got, want, c.origin) + } + } + + c.addAction(func([]interface{}) []interface{} { + return rets + }) + + return c +} + +// Times declares the exact number of times a function call is expected to be executed. +func (c *Call) Times(n int) *Call { + c.minCalls, c.maxCalls = n, n + return c +} + +// SetArg declares an action that will set the nth argument's value, +// indirected through a pointer. Or, in the case of a slice, SetArg +// will copy value's elements into the nth argument. +func (c *Call) SetArg(n int, value interface{}) *Call { + c.t.Helper() + + mt := c.methodType + // TODO: This will break on variadic methods. + // We will need to check those at invocation time. + if n < 0 || n >= mt.NumIn() { + c.t.Fatalf("SetArg(%d, ...) called for a method with %d args [%s]", + n, mt.NumIn(), c.origin) + } + // Permit setting argument through an interface. + // In the interface case, we don't (nay, can't) check the type here. + at := mt.In(n) + switch at.Kind() { + case reflect.Ptr: + dt := at.Elem() + if vt := reflect.TypeOf(value); !vt.AssignableTo(dt) { + c.t.Fatalf("SetArg(%d, ...) argument is a %v, not assignable to %v [%s]", + n, vt, dt, c.origin) + } + case reflect.Interface: + // nothing to do + case reflect.Slice: + // nothing to do + default: + c.t.Fatalf("SetArg(%d, ...) referring to argument of non-pointer non-interface non-slice type %v [%s]", + n, at, c.origin) + } + + c.addAction(func(args []interface{}) []interface{} { + v := reflect.ValueOf(value) + switch reflect.TypeOf(args[n]).Kind() { + case reflect.Slice: + setSlice(args[n], v) + default: + reflect.ValueOf(args[n]).Elem().Set(v) + } + return nil + }) + return c +} + +// isPreReq returns true if other is a direct or indirect prerequisite to c. +func (c *Call) isPreReq(other *Call) bool { + for _, preReq := range c.preReqs { + if other == preReq || preReq.isPreReq(other) { + return true + } + } + return false +} + +// After declares that the call may only match after preReq has been exhausted. +func (c *Call) After(preReq *Call) *Call { + c.t.Helper() + + if c == preReq { + c.t.Fatalf("A call isn't allowed to be its own prerequisite") + } + if preReq.isPreReq(c) { + c.t.Fatalf("Loop in call order: %v is a prerequisite to %v (possibly indirectly).", c, preReq) + } + + c.preReqs = append(c.preReqs, preReq) + return c +} + +// Returns true if the minimum number of calls have been made. +func (c *Call) satisfied() bool { + return c.numCalls >= c.minCalls +} + +// Returns true if the maximum number of calls have been made. +func (c *Call) exhausted() bool { + return c.numCalls >= c.maxCalls +} + +func (c *Call) String() string { + args := make([]string, len(c.args)) + for i, arg := range c.args { + args[i] = arg.String() + } + arguments := strings.Join(args, ", ") + return fmt.Sprintf("%T.%v(%s) %s", c.receiver, c.method, arguments, c.origin) +} + +// Tests if the given call matches the expected call. +// If yes, returns nil. If no, returns error with message explaining why it does not match. +func (c *Call) matches(args []interface{}) error { + if !c.methodType.IsVariadic() { + if len(args) != len(c.args) { + return fmt.Errorf("expected call at %s has the wrong number of arguments. Got: %d, want: %d", + c.origin, len(args), len(c.args)) + } + + for i, m := range c.args { + if !m.Matches(args[i]) { + return fmt.Errorf( + "expected call at %s doesn't match the argument at index %d.\nGot: %v\nWant: %v", + c.origin, i, formatGottenArg(m, args[i]), m, + ) + } + } + } else { + if len(c.args) < c.methodType.NumIn()-1 { + return fmt.Errorf("expected call at %s has the wrong number of matchers. Got: %d, want: %d", + c.origin, len(c.args), c.methodType.NumIn()-1) + } + if len(c.args) != c.methodType.NumIn() && len(args) != len(c.args) { + return fmt.Errorf("expected call at %s has the wrong number of arguments. Got: %d, want: %d", + c.origin, len(args), len(c.args)) + } + if len(args) < len(c.args)-1 { + return fmt.Errorf("expected call at %s has the wrong number of arguments. Got: %d, want: greater than or equal to %d", + c.origin, len(args), len(c.args)-1) + } + + for i, m := range c.args { + if i < c.methodType.NumIn()-1 { + // Non-variadic args + if !m.Matches(args[i]) { + return fmt.Errorf("expected call at %s doesn't match the argument at index %s.\nGot: %v\nWant: %v", + c.origin, strconv.Itoa(i), formatGottenArg(m, args[i]), m) + } + continue + } + // The last arg has a possibility of a variadic argument, so let it branch + + // sample: Foo(a int, b int, c ...int) + if i < len(c.args) && i < len(args) { + if m.Matches(args[i]) { + // Got Foo(a, b, c) want Foo(matcherA, matcherB, gomock.Any()) + // Got Foo(a, b, c) want Foo(matcherA, matcherB, someSliceMatcher) + // Got Foo(a, b, c) want Foo(matcherA, matcherB, matcherC) + // Got Foo(a, b) want Foo(matcherA, matcherB) + // Got Foo(a, b, c, d) want Foo(matcherA, matcherB, matcherC, matcherD) + continue + } + } + + // The number of actual args don't match the number of matchers, + // or the last matcher is a slice and the last arg is not. + // If this function still matches it is because the last matcher + // matches all the remaining arguments or the lack of any. + // Convert the remaining arguments, if any, into a slice of the + // expected type. + vArgsType := c.methodType.In(c.methodType.NumIn() - 1) + vArgs := reflect.MakeSlice(vArgsType, 0, len(args)-i) + for _, arg := range args[i:] { + vArgs = reflect.Append(vArgs, reflect.ValueOf(arg)) + } + if m.Matches(vArgs.Interface()) { + // Got Foo(a, b, c, d, e) want Foo(matcherA, matcherB, gomock.Any()) + // Got Foo(a, b, c, d, e) want Foo(matcherA, matcherB, someSliceMatcher) + // Got Foo(a, b) want Foo(matcherA, matcherB, gomock.Any()) + // Got Foo(a, b) want Foo(matcherA, matcherB, someEmptySliceMatcher) + break + } + // Wrong number of matchers or not match. Fail. + // Got Foo(a, b) want Foo(matcherA, matcherB, matcherC, matcherD) + // Got Foo(a, b, c) want Foo(matcherA, matcherB, matcherC, matcherD) + // Got Foo(a, b, c, d) want Foo(matcherA, matcherB, matcherC, matcherD, matcherE) + // Got Foo(a, b, c, d, e) want Foo(matcherA, matcherB, matcherC, matcherD) + // Got Foo(a, b, c) want Foo(matcherA, matcherB) + + return fmt.Errorf("expected call at %s doesn't match the argument at index %s.\nGot: %v\nWant: %v", + c.origin, strconv.Itoa(i), formatGottenArg(m, args[i:]), c.args[i]) + } + } + + // Check that all prerequisite calls have been satisfied. + for _, preReqCall := range c.preReqs { + if !preReqCall.satisfied() { + return fmt.Errorf("expected call at %s doesn't have a prerequisite call satisfied:\n%v\nshould be called before:\n%v", + c.origin, preReqCall, c) + } + } + + // Check that the call is not exhausted. + if c.exhausted() { + return fmt.Errorf("expected call at %s has already been called the max number of times", c.origin) + } + + return nil +} + +// dropPrereqs tells the expected Call to not re-check prerequisite calls any +// longer, and to return its current set. +func (c *Call) dropPrereqs() (preReqs []*Call) { + preReqs = c.preReqs + c.preReqs = nil + return +} + +func (c *Call) call() []func([]interface{}) []interface{} { + c.numCalls++ + return c.actions +} + +// InOrder declares that the given calls should occur in order. +func InOrder(calls ...*Call) { + for i := 1; i < len(calls); i++ { + calls[i].After(calls[i-1]) + } +} + +func setSlice(arg interface{}, v reflect.Value) { + va := reflect.ValueOf(arg) + for i := 0; i < v.Len(); i++ { + va.Index(i).Set(v.Index(i)) + } +} + +func (c *Call) addAction(action func([]interface{}) []interface{}) { + c.actions = append(c.actions, action) +} + +func formatGottenArg(m Matcher, arg interface{}) string { + got := fmt.Sprintf("%v (%T)", arg, arg) + if gs, ok := m.(GotFormatter); ok { + got = gs.Got(arg) + } + return got +} diff --git a/vendor/github.com/golang/mock/gomock/callset.go b/vendor/github.com/golang/mock/gomock/callset.go new file mode 100644 index 00000000000..49dba787a40 --- /dev/null +++ b/vendor/github.com/golang/mock/gomock/callset.go @@ -0,0 +1,113 @@ +// Copyright 2011 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package gomock + +import ( + "bytes" + "errors" + "fmt" +) + +// callSet represents a set of expected calls, indexed by receiver and method +// name. +type callSet struct { + // Calls that are still expected. + expected map[callSetKey][]*Call + // Calls that have been exhausted. + exhausted map[callSetKey][]*Call +} + +// callSetKey is the key in the maps in callSet +type callSetKey struct { + receiver interface{} + fname string +} + +func newCallSet() *callSet { + return &callSet{make(map[callSetKey][]*Call), make(map[callSetKey][]*Call)} +} + +// Add adds a new expected call. +func (cs callSet) Add(call *Call) { + key := callSetKey{call.receiver, call.method} + m := cs.expected + if call.exhausted() { + m = cs.exhausted + } + m[key] = append(m[key], call) +} + +// Remove removes an expected call. +func (cs callSet) Remove(call *Call) { + key := callSetKey{call.receiver, call.method} + calls := cs.expected[key] + for i, c := range calls { + if c == call { + // maintain order for remaining calls + cs.expected[key] = append(calls[:i], calls[i+1:]...) + cs.exhausted[key] = append(cs.exhausted[key], call) + break + } + } +} + +// FindMatch searches for a matching call. Returns error with explanation message if no call matched. +func (cs callSet) FindMatch(receiver interface{}, method string, args []interface{}) (*Call, error) { + key := callSetKey{receiver, method} + + // Search through the expected calls. + expected := cs.expected[key] + var callsErrors bytes.Buffer + for _, call := range expected { + err := call.matches(args) + if err != nil { + _, _ = fmt.Fprintf(&callsErrors, "\n%v", err) + } else { + return call, nil + } + } + + // If we haven't found a match then search through the exhausted calls so we + // get useful error messages. + exhausted := cs.exhausted[key] + for _, call := range exhausted { + if err := call.matches(args); err != nil { + _, _ = fmt.Fprintf(&callsErrors, "\n%v", err) + continue + } + _, _ = fmt.Fprintf( + &callsErrors, "all expected calls for method %q have been exhausted", method, + ) + } + + if len(expected)+len(exhausted) == 0 { + _, _ = fmt.Fprintf(&callsErrors, "there are no expected calls of the method %q for that receiver", method) + } + + return nil, errors.New(callsErrors.String()) +} + +// Failures returns the calls that are not satisfied. +func (cs callSet) Failures() []*Call { + failures := make([]*Call, 0, len(cs.expected)) + for _, calls := range cs.expected { + for _, call := range calls { + if !call.satisfied() { + failures = append(failures, call) + } + } + } + return failures +} diff --git a/vendor/github.com/golang/mock/gomock/controller.go b/vendor/github.com/golang/mock/gomock/controller.go new file mode 100644 index 00000000000..f054200d56c --- /dev/null +++ b/vendor/github.com/golang/mock/gomock/controller.go @@ -0,0 +1,336 @@ +// Copyright 2010 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package gomock is a mock framework for Go. +// +// Standard usage: +// (1) Define an interface that you wish to mock. +// type MyInterface interface { +// SomeMethod(x int64, y string) +// } +// (2) Use mockgen to generate a mock from the interface. +// (3) Use the mock in a test: +// func TestMyThing(t *testing.T) { +// mockCtrl := gomock.NewController(t) +// defer mockCtrl.Finish() +// +// mockObj := something.NewMockMyInterface(mockCtrl) +// mockObj.EXPECT().SomeMethod(4, "blah") +// // pass mockObj to a real object and play with it. +// } +// +// By default, expected calls are not enforced to run in any particular order. +// Call order dependency can be enforced by use of InOrder and/or Call.After. +// Call.After can create more varied call order dependencies, but InOrder is +// often more convenient. +// +// The following examples create equivalent call order dependencies. +// +// Example of using Call.After to chain expected call order: +// +// firstCall := mockObj.EXPECT().SomeMethod(1, "first") +// secondCall := mockObj.EXPECT().SomeMethod(2, "second").After(firstCall) +// mockObj.EXPECT().SomeMethod(3, "third").After(secondCall) +// +// Example of using InOrder to declare expected call order: +// +// gomock.InOrder( +// mockObj.EXPECT().SomeMethod(1, "first"), +// mockObj.EXPECT().SomeMethod(2, "second"), +// mockObj.EXPECT().SomeMethod(3, "third"), +// ) +package gomock + +import ( + "context" + "fmt" + "reflect" + "runtime" + "sync" +) + +// A TestReporter is something that can be used to report test failures. It +// is satisfied by the standard library's *testing.T. +type TestReporter interface { + Errorf(format string, args ...interface{}) + Fatalf(format string, args ...interface{}) +} + +// TestHelper is a TestReporter that has the Helper method. It is satisfied +// by the standard library's *testing.T. +type TestHelper interface { + TestReporter + Helper() +} + +// cleanuper is used to check if TestHelper also has the `Cleanup` method. A +// common pattern is to pass in a `*testing.T` to +// `NewController(t TestReporter)`. In Go 1.14+, `*testing.T` has a cleanup +// method. This can be utilized to call `Finish()` so the caller of this library +// does not have to. +type cleanuper interface { + Cleanup(func()) +} + +// A Controller represents the top-level control of a mock ecosystem. It +// defines the scope and lifetime of mock objects, as well as their +// expectations. It is safe to call Controller's methods from multiple +// goroutines. Each test should create a new Controller and invoke Finish via +// defer. +// +// func TestFoo(t *testing.T) { +// ctrl := gomock.NewController(t) +// defer ctrl.Finish() +// // .. +// } +// +// func TestBar(t *testing.T) { +// t.Run("Sub-Test-1", st) { +// ctrl := gomock.NewController(st) +// defer ctrl.Finish() +// // .. +// }) +// t.Run("Sub-Test-2", st) { +// ctrl := gomock.NewController(st) +// defer ctrl.Finish() +// // .. +// }) +// }) +type Controller struct { + // T should only be called within a generated mock. It is not intended to + // be used in user code and may be changed in future versions. T is the + // TestReporter passed in when creating the Controller via NewController. + // If the TestReporter does not implement a TestHelper it will be wrapped + // with a nopTestHelper. + T TestHelper + mu sync.Mutex + expectedCalls *callSet + finished bool +} + +// NewController returns a new Controller. It is the preferred way to create a +// Controller. +// +// New in go1.14+, if you are passing a *testing.T into this function you no +// longer need to call ctrl.Finish() in your test methods. +func NewController(t TestReporter) *Controller { + h, ok := t.(TestHelper) + if !ok { + h = &nopTestHelper{t} + } + ctrl := &Controller{ + T: h, + expectedCalls: newCallSet(), + } + if c, ok := isCleanuper(ctrl.T); ok { + c.Cleanup(func() { + ctrl.T.Helper() + ctrl.finish(true, nil) + }) + } + + return ctrl +} + +type cancelReporter struct { + t TestHelper + cancel func() +} + +func (r *cancelReporter) Errorf(format string, args ...interface{}) { + r.t.Errorf(format, args...) +} +func (r *cancelReporter) Fatalf(format string, args ...interface{}) { + defer r.cancel() + r.t.Fatalf(format, args...) +} + +func (r *cancelReporter) Helper() { + r.t.Helper() +} + +// WithContext returns a new Controller and a Context, which is cancelled on any +// fatal failure. +func WithContext(ctx context.Context, t TestReporter) (*Controller, context.Context) { + h, ok := t.(TestHelper) + if !ok { + h = &nopTestHelper{t: t} + } + + ctx, cancel := context.WithCancel(ctx) + return NewController(&cancelReporter{t: h, cancel: cancel}), ctx +} + +type nopTestHelper struct { + t TestReporter +} + +func (h *nopTestHelper) Errorf(format string, args ...interface{}) { + h.t.Errorf(format, args...) +} +func (h *nopTestHelper) Fatalf(format string, args ...interface{}) { + h.t.Fatalf(format, args...) +} + +func (h nopTestHelper) Helper() {} + +// RecordCall is called by a mock. It should not be called by user code. +func (ctrl *Controller) RecordCall(receiver interface{}, method string, args ...interface{}) *Call { + ctrl.T.Helper() + + recv := reflect.ValueOf(receiver) + for i := 0; i < recv.Type().NumMethod(); i++ { + if recv.Type().Method(i).Name == method { + return ctrl.RecordCallWithMethodType(receiver, method, recv.Method(i).Type(), args...) + } + } + ctrl.T.Fatalf("gomock: failed finding method %s on %T", method, receiver) + panic("unreachable") +} + +// RecordCallWithMethodType is called by a mock. It should not be called by user code. +func (ctrl *Controller) RecordCallWithMethodType(receiver interface{}, method string, methodType reflect.Type, args ...interface{}) *Call { + ctrl.T.Helper() + + call := newCall(ctrl.T, receiver, method, methodType, args...) + + ctrl.mu.Lock() + defer ctrl.mu.Unlock() + ctrl.expectedCalls.Add(call) + + return call +} + +// Call is called by a mock. It should not be called by user code. +func (ctrl *Controller) Call(receiver interface{}, method string, args ...interface{}) []interface{} { + ctrl.T.Helper() + + // Nest this code so we can use defer to make sure the lock is released. + actions := func() []func([]interface{}) []interface{} { + ctrl.T.Helper() + ctrl.mu.Lock() + defer ctrl.mu.Unlock() + + expected, err := ctrl.expectedCalls.FindMatch(receiver, method, args) + if err != nil { + // callerInfo's skip should be updated if the number of calls between the user's test + // and this line changes, i.e. this code is wrapped in another anonymous function. + // 0 is us, 1 is controller.Call(), 2 is the generated mock, and 3 is the user's test. + origin := callerInfo(3) + ctrl.T.Fatalf("Unexpected call to %T.%v(%v) at %s because: %s", receiver, method, args, origin, err) + } + + // Two things happen here: + // * the matching call no longer needs to check prerequite calls, + // * and the prerequite calls are no longer expected, so remove them. + preReqCalls := expected.dropPrereqs() + for _, preReqCall := range preReqCalls { + ctrl.expectedCalls.Remove(preReqCall) + } + + actions := expected.call() + if expected.exhausted() { + ctrl.expectedCalls.Remove(expected) + } + return actions + }() + + var rets []interface{} + for _, action := range actions { + if r := action(args); r != nil { + rets = r + } + } + + return rets +} + +// Finish checks to see if all the methods that were expected to be called +// were called. It should be invoked for each Controller. It is not idempotent +// and therefore can only be invoked once. +// +// New in go1.14+, if you are passing a *testing.T into NewController function you no +// longer need to call ctrl.Finish() in your test methods. +func (ctrl *Controller) Finish() { + // If we're currently panicking, probably because this is a deferred call. + // This must be recovered in the deferred function. + err := recover() + ctrl.finish(false, err) +} + +func (ctrl *Controller) finish(cleanup bool, panicErr interface{}) { + ctrl.T.Helper() + + ctrl.mu.Lock() + defer ctrl.mu.Unlock() + + if ctrl.finished { + if _, ok := isCleanuper(ctrl.T); !ok { + ctrl.T.Fatalf("Controller.Finish was called more than once. It has to be called exactly once.") + } + return + } + ctrl.finished = true + + // Short-circuit, pass through the panic. + if panicErr != nil { + panic(panicErr) + } + + // Check that all remaining expected calls are satisfied. + failures := ctrl.expectedCalls.Failures() + for _, call := range failures { + ctrl.T.Errorf("missing call(s) to %v", call) + } + if len(failures) != 0 { + if !cleanup { + ctrl.T.Fatalf("aborting test due to missing call(s)") + return + } + ctrl.T.Errorf("aborting test due to missing call(s)") + } +} + +// callerInfo returns the file:line of the call site. skip is the number +// of stack frames to skip when reporting. 0 is callerInfo's call site. +func callerInfo(skip int) string { + if _, file, line, ok := runtime.Caller(skip + 1); ok { + return fmt.Sprintf("%s:%d", file, line) + } + return "unknown file" +} + +// isCleanuper checks it if t's base TestReporter has a Cleanup method. +func isCleanuper(t TestReporter) (cleanuper, bool) { + tr := unwrapTestReporter(t) + c, ok := tr.(cleanuper) + return c, ok +} + +// unwrapTestReporter unwraps TestReporter to the base implementation. +func unwrapTestReporter(t TestReporter) TestReporter { + tr := t + switch nt := t.(type) { + case *cancelReporter: + tr = nt.t + if h, check := tr.(*nopTestHelper); check { + tr = h.t + } + case *nopTestHelper: + tr = nt.t + default: + // not wrapped + } + return tr +} diff --git a/vendor/github.com/golang/mock/gomock/matchers.go b/vendor/github.com/golang/mock/gomock/matchers.go new file mode 100644 index 00000000000..2822fb2c8c4 --- /dev/null +++ b/vendor/github.com/golang/mock/gomock/matchers.go @@ -0,0 +1,341 @@ +// Copyright 2010 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package gomock + +import ( + "fmt" + "reflect" + "strings" +) + +// A Matcher is a representation of a class of values. +// It is used to represent the valid or expected arguments to a mocked method. +type Matcher interface { + // Matches returns whether x is a match. + Matches(x interface{}) bool + + // String describes what the matcher matches. + String() string +} + +// WantFormatter modifies the given Matcher's String() method to the given +// Stringer. This allows for control on how the "Want" is formatted when +// printing . +func WantFormatter(s fmt.Stringer, m Matcher) Matcher { + type matcher interface { + Matches(x interface{}) bool + } + + return struct { + matcher + fmt.Stringer + }{ + matcher: m, + Stringer: s, + } +} + +// StringerFunc type is an adapter to allow the use of ordinary functions as +// a Stringer. If f is a function with the appropriate signature, +// StringerFunc(f) is a Stringer that calls f. +type StringerFunc func() string + +// String implements fmt.Stringer. +func (f StringerFunc) String() string { + return f() +} + +// GotFormatter is used to better print failure messages. If a matcher +// implements GotFormatter, it will use the result from Got when printing +// the failure message. +type GotFormatter interface { + // Got is invoked with the received value. The result is used when + // printing the failure message. + Got(got interface{}) string +} + +// GotFormatterFunc type is an adapter to allow the use of ordinary +// functions as a GotFormatter. If f is a function with the appropriate +// signature, GotFormatterFunc(f) is a GotFormatter that calls f. +type GotFormatterFunc func(got interface{}) string + +// Got implements GotFormatter. +func (f GotFormatterFunc) Got(got interface{}) string { + return f(got) +} + +// GotFormatterAdapter attaches a GotFormatter to a Matcher. +func GotFormatterAdapter(s GotFormatter, m Matcher) Matcher { + return struct { + GotFormatter + Matcher + }{ + GotFormatter: s, + Matcher: m, + } +} + +type anyMatcher struct{} + +func (anyMatcher) Matches(interface{}) bool { + return true +} + +func (anyMatcher) String() string { + return "is anything" +} + +type eqMatcher struct { + x interface{} +} + +func (e eqMatcher) Matches(x interface{}) bool { + // In case, some value is nil + if e.x == nil || x == nil { + return reflect.DeepEqual(e.x, x) + } + + // Check if types assignable and convert them to common type + x1Val := reflect.ValueOf(e.x) + x2Val := reflect.ValueOf(x) + + if x1Val.Type().AssignableTo(x2Val.Type()) { + x1ValConverted := x1Val.Convert(x2Val.Type()) + return reflect.DeepEqual(x1ValConverted.Interface(), x2Val.Interface()) + } + + return false +} + +func (e eqMatcher) String() string { + return fmt.Sprintf("is equal to %v (%T)", e.x, e.x) +} + +type nilMatcher struct{} + +func (nilMatcher) Matches(x interface{}) bool { + if x == nil { + return true + } + + v := reflect.ValueOf(x) + switch v.Kind() { + case reflect.Chan, reflect.Func, reflect.Interface, reflect.Map, + reflect.Ptr, reflect.Slice: + return v.IsNil() + } + + return false +} + +func (nilMatcher) String() string { + return "is nil" +} + +type notMatcher struct { + m Matcher +} + +func (n notMatcher) Matches(x interface{}) bool { + return !n.m.Matches(x) +} + +func (n notMatcher) String() string { + return "not(" + n.m.String() + ")" +} + +type assignableToTypeOfMatcher struct { + targetType reflect.Type +} + +func (m assignableToTypeOfMatcher) Matches(x interface{}) bool { + return reflect.TypeOf(x).AssignableTo(m.targetType) +} + +func (m assignableToTypeOfMatcher) String() string { + return "is assignable to " + m.targetType.Name() +} + +type allMatcher struct { + matchers []Matcher +} + +func (am allMatcher) Matches(x interface{}) bool { + for _, m := range am.matchers { + if !m.Matches(x) { + return false + } + } + return true +} + +func (am allMatcher) String() string { + ss := make([]string, 0, len(am.matchers)) + for _, matcher := range am.matchers { + ss = append(ss, matcher.String()) + } + return strings.Join(ss, "; ") +} + +type lenMatcher struct { + i int +} + +func (m lenMatcher) Matches(x interface{}) bool { + v := reflect.ValueOf(x) + switch v.Kind() { + case reflect.Array, reflect.Chan, reflect.Map, reflect.Slice, reflect.String: + return v.Len() == m.i + default: + return false + } +} + +func (m lenMatcher) String() string { + return fmt.Sprintf("has length %d", m.i) +} + +type inAnyOrderMatcher struct { + x interface{} +} + +func (m inAnyOrderMatcher) Matches(x interface{}) bool { + given, ok := m.prepareValue(x) + if !ok { + return false + } + wanted, ok := m.prepareValue(m.x) + if !ok { + return false + } + + if given.Len() != wanted.Len() { + return false + } + + usedFromGiven := make([]bool, given.Len()) + foundFromWanted := make([]bool, wanted.Len()) + for i := 0; i < wanted.Len(); i++ { + wantedMatcher := Eq(wanted.Index(i).Interface()) + for j := 0; j < given.Len(); j++ { + if usedFromGiven[j] { + continue + } + if wantedMatcher.Matches(given.Index(j).Interface()) { + foundFromWanted[i] = true + usedFromGiven[j] = true + break + } + } + } + + missingFromWanted := 0 + for _, found := range foundFromWanted { + if !found { + missingFromWanted++ + } + } + extraInGiven := 0 + for _, used := range usedFromGiven { + if !used { + extraInGiven++ + } + } + + return extraInGiven == 0 && missingFromWanted == 0 +} + +func (m inAnyOrderMatcher) prepareValue(x interface{}) (reflect.Value, bool) { + xValue := reflect.ValueOf(x) + switch xValue.Kind() { + case reflect.Slice, reflect.Array: + return xValue, true + default: + return reflect.Value{}, false + } +} + +func (m inAnyOrderMatcher) String() string { + return fmt.Sprintf("has the same elements as %v", m.x) +} + +// Constructors + +// All returns a composite Matcher that returns true if and only all of the +// matchers return true. +func All(ms ...Matcher) Matcher { return allMatcher{ms} } + +// Any returns a matcher that always matches. +func Any() Matcher { return anyMatcher{} } + +// Eq returns a matcher that matches on equality. +// +// Example usage: +// Eq(5).Matches(5) // returns true +// Eq(5).Matches(4) // returns false +func Eq(x interface{}) Matcher { return eqMatcher{x} } + +// Len returns a matcher that matches on length. This matcher returns false if +// is compared to a type that is not an array, chan, map, slice, or string. +func Len(i int) Matcher { + return lenMatcher{i} +} + +// Nil returns a matcher that matches if the received value is nil. +// +// Example usage: +// var x *bytes.Buffer +// Nil().Matches(x) // returns true +// x = &bytes.Buffer{} +// Nil().Matches(x) // returns false +func Nil() Matcher { return nilMatcher{} } + +// Not reverses the results of its given child matcher. +// +// Example usage: +// Not(Eq(5)).Matches(4) // returns true +// Not(Eq(5)).Matches(5) // returns false +func Not(x interface{}) Matcher { + if m, ok := x.(Matcher); ok { + return notMatcher{m} + } + return notMatcher{Eq(x)} +} + +// AssignableToTypeOf is a Matcher that matches if the parameter to the mock +// function is assignable to the type of the parameter to this function. +// +// Example usage: +// var s fmt.Stringer = &bytes.Buffer{} +// AssignableToTypeOf(s).Matches(time.Second) // returns true +// AssignableToTypeOf(s).Matches(99) // returns false +// +// var ctx = reflect.TypeOf((*context.Context)(nil)).Elem() +// AssignableToTypeOf(ctx).Matches(context.Background()) // returns true +func AssignableToTypeOf(x interface{}) Matcher { + if xt, ok := x.(reflect.Type); ok { + return assignableToTypeOfMatcher{xt} + } + return assignableToTypeOfMatcher{reflect.TypeOf(x)} +} + +// InAnyOrder is a Matcher that returns true for collections of the same elements ignoring the order. +// +// Example usage: +// InAnyOrder([]int{1, 2, 3}).Matches([]int{1, 3, 2}) // returns true +// InAnyOrder([]int{1, 2, 3}).Matches([]int{1, 2}) // returns false +func InAnyOrder(x interface{}) Matcher { + return inAnyOrderMatcher{x} +} diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/backoff.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/backoff.go new file mode 100644 index 00000000000..ad35f09a87f --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/backoff.go @@ -0,0 +1,44 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +package grpc_retry + +import ( + "time" + + "github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils" +) + +// BackoffLinear is very simple: it waits for a fixed period of time between calls. +func BackoffLinear(waitBetween time.Duration) BackoffFunc { + return func(attempt uint) time.Duration { + return waitBetween + } +} + +// BackoffLinearWithJitter waits a set period of time, allowing for jitter (fractional adjustment). +// +// For example waitBetween=1s and jitter=0.10 can generate waits between 900ms and 1100ms. +func BackoffLinearWithJitter(waitBetween time.Duration, jitterFraction float64) BackoffFunc { + return func(attempt uint) time.Duration { + return backoffutils.JitterUp(waitBetween, jitterFraction) + } +} + +// BackoffExponential produces increasing intervals for each attempt. +// +// The scalar is multiplied times 2 raised to the current attempt. So the first +// retry with a scalar of 100ms is 100ms, while the 5th attempt would be 1.6s. +func BackoffExponential(scalar time.Duration) BackoffFunc { + return func(attempt uint) time.Duration { + return scalar * time.Duration(backoffutils.ExponentBase2(attempt)) + } +} + +// BackoffExponentialWithJitter creates an exponential backoff like +// BackoffExponential does, but adds jitter. +func BackoffExponentialWithJitter(scalar time.Duration, jitterFraction float64) BackoffFunc { + return func(attempt uint) time.Duration { + return backoffutils.JitterUp(scalar*time.Duration(backoffutils.ExponentBase2(attempt)), jitterFraction) + } +} diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/doc.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/doc.go new file mode 100644 index 00000000000..f8ba7198a5a --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/doc.go @@ -0,0 +1,25 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +/* +`grpc_retry` provides client-side request retry logic for gRPC. + +Client-Side Request Retry Interceptor + +It allows for automatic retry, inside the generated gRPC code of requests based on the gRPC status +of the reply. It supports unary (1:1), and server stream (1:n) requests. + +By default the interceptors *are disabled*, preventing accidental use of retries. You can easily +override the number of retries (setting them to more than 0) with a `grpc.ClientOption`, e.g.: + + myclient.Ping(ctx, goodPing, grpc_retry.WithMax(5)) + +Other default options are: retry on `ResourceExhausted` and `Unavailable` gRPC codes, use a 50ms +linear backoff with 10% jitter. + +For chained interceptors, the retry interceptor will call every interceptor that follows it +whenever a retry happens. + +Please see examples for more advanced use. +*/ +package grpc_retry diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/options.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/options.go new file mode 100644 index 00000000000..7a633e29347 --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/options.go @@ -0,0 +1,142 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +package grpc_retry + +import ( + "context" + "time" + + "google.golang.org/grpc" + "google.golang.org/grpc/codes" +) + +var ( + // DefaultRetriableCodes is a set of well known types gRPC codes that should be retri-able. + // + // `ResourceExhausted` means that the user quota, e.g. per-RPC limits, have been reached. + // `Unavailable` means that system is currently unavailable and the client should retry again. + DefaultRetriableCodes = []codes.Code{codes.ResourceExhausted, codes.Unavailable} + + defaultOptions = &options{ + max: 0, // disabled + perCallTimeout: 0, // disabled + includeHeader: true, + codes: DefaultRetriableCodes, + backoffFunc: BackoffFuncContext(func(ctx context.Context, attempt uint) time.Duration { + return BackoffLinearWithJitter(50*time.Millisecond /*jitter*/, 0.10)(attempt) + }), + } +) + +// BackoffFunc denotes a family of functions that control the backoff duration between call retries. +// +// They are called with an identifier of the attempt, and should return a time the system client should +// hold off for. If the time returned is longer than the `context.Context.Deadline` of the request +// the deadline of the request takes precedence and the wait will be interrupted before proceeding +// with the next iteration. +type BackoffFunc func(attempt uint) time.Duration + +// BackoffFuncContext denotes a family of functions that control the backoff duration between call retries. +// +// They are called with an identifier of the attempt, and should return a time the system client should +// hold off for. If the time returned is longer than the `context.Context.Deadline` of the request +// the deadline of the request takes precedence and the wait will be interrupted before proceeding +// with the next iteration. The context can be used to extract request scoped metadata and context values. +type BackoffFuncContext func(ctx context.Context, attempt uint) time.Duration + +// Disable disables the retry behaviour on this call, or this interceptor. +// +// Its semantically the same to `WithMax` +func Disable() CallOption { + return WithMax(0) +} + +// WithMax sets the maximum number of retries on this call, or this interceptor. +func WithMax(maxRetries uint) CallOption { + return CallOption{applyFunc: func(o *options) { + o.max = maxRetries + }} +} + +// WithBackoff sets the `BackoffFunc` used to control time between retries. +func WithBackoff(bf BackoffFunc) CallOption { + return CallOption{applyFunc: func(o *options) { + o.backoffFunc = BackoffFuncContext(func(ctx context.Context, attempt uint) time.Duration { + return bf(attempt) + }) + }} +} + +// WithBackoffContext sets the `BackoffFuncContext` used to control time between retries. +func WithBackoffContext(bf BackoffFuncContext) CallOption { + return CallOption{applyFunc: func(o *options) { + o.backoffFunc = bf + }} +} + +// WithCodes sets which codes should be retried. +// +// Please *use with care*, as you may be retrying non-idempotent calls. +// +// You cannot automatically retry on Cancelled and Deadline, please use `WithPerRetryTimeout` for these. +func WithCodes(retryCodes ...codes.Code) CallOption { + return CallOption{applyFunc: func(o *options) { + o.codes = retryCodes + }} +} + +// WithPerRetryTimeout sets the RPC timeout per call (including initial call) on this call, or this interceptor. +// +// The context.Deadline of the call takes precedence and sets the maximum time the whole invocation +// will take, but WithPerRetryTimeout can be used to limit the RPC time per each call. +// +// For example, with context.Deadline = now + 10s, and WithPerRetryTimeout(3 * time.Seconds), each +// of the retry calls (including the initial one) will have a deadline of now + 3s. +// +// A value of 0 disables the timeout overrides completely and returns to each retry call using the +// parent `context.Deadline`. +// +// Note that when this is enabled, any DeadlineExceeded errors that are propagated up will be retried. +func WithPerRetryTimeout(timeout time.Duration) CallOption { + return CallOption{applyFunc: func(o *options) { + o.perCallTimeout = timeout + }} +} + +type options struct { + max uint + perCallTimeout time.Duration + includeHeader bool + codes []codes.Code + backoffFunc BackoffFuncContext +} + +// CallOption is a grpc.CallOption that is local to grpc_retry. +type CallOption struct { + grpc.EmptyCallOption // make sure we implement private after() and before() fields so we don't panic. + applyFunc func(opt *options) +} + +func reuseOrNewWithCallOptions(opt *options, callOptions []CallOption) *options { + if len(callOptions) == 0 { + return opt + } + optCopy := &options{} + *optCopy = *opt + for _, f := range callOptions { + f.applyFunc(optCopy) + } + return optCopy +} + +func filterCallOptions(callOptions []grpc.CallOption) (grpcOptions []grpc.CallOption, retryOptions []CallOption) { + for _, opt := range callOptions { + if co, ok := opt.(CallOption); ok { + retryOptions = append(retryOptions, co) + } else { + grpcOptions = append(grpcOptions, opt) + } + } + return grpcOptions, retryOptions +} diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/retry.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/retry.go new file mode 100644 index 00000000000..003bbd9066e --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/retry/retry.go @@ -0,0 +1,319 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +package grpc_retry + +import ( + "context" + "io" + "strconv" + "sync" + "time" + + "github.com/grpc-ecosystem/go-grpc-middleware/util/metautils" + "golang.org/x/net/trace" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" +) + +const ( + AttemptMetadataKey = "x-retry-attempty" +) + +// UnaryClientInterceptor returns a new retrying unary client interceptor. +// +// The default configuration of the interceptor is to not retry *at all*. This behaviour can be +// changed through options (e.g. WithMax) on creation of the interceptor or on call (through grpc.CallOptions). +func UnaryClientInterceptor(optFuncs ...CallOption) grpc.UnaryClientInterceptor { + intOpts := reuseOrNewWithCallOptions(defaultOptions, optFuncs) + return func(parentCtx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + grpcOpts, retryOpts := filterCallOptions(opts) + callOpts := reuseOrNewWithCallOptions(intOpts, retryOpts) + // short circuit for simplicity, and avoiding allocations. + if callOpts.max == 0 { + return invoker(parentCtx, method, req, reply, cc, grpcOpts...) + } + var lastErr error + for attempt := uint(0); attempt < callOpts.max; attempt++ { + if err := waitRetryBackoff(attempt, parentCtx, callOpts); err != nil { + return err + } + callCtx := perCallContext(parentCtx, callOpts, attempt) + lastErr = invoker(callCtx, method, req, reply, cc, grpcOpts...) + // TODO(mwitkow): Maybe dial and transport errors should be retriable? + if lastErr == nil { + return nil + } + logTrace(parentCtx, "grpc_retry attempt: %d, got err: %v", attempt, lastErr) + if isContextError(lastErr) { + if parentCtx.Err() != nil { + logTrace(parentCtx, "grpc_retry attempt: %d, parent context error: %v", attempt, parentCtx.Err()) + // its the parent context deadline or cancellation. + return lastErr + } else if callOpts.perCallTimeout != 0 { + // We have set a perCallTimeout in the retry middleware, which would result in a context error if + // the deadline was exceeded, in which case try again. + logTrace(parentCtx, "grpc_retry attempt: %d, context error from retry call", attempt) + continue + } + } + if !isRetriable(lastErr, callOpts) { + return lastErr + } + } + return lastErr + } +} + +// StreamClientInterceptor returns a new retrying stream client interceptor for server side streaming calls. +// +// The default configuration of the interceptor is to not retry *at all*. This behaviour can be +// changed through options (e.g. WithMax) on creation of the interceptor or on call (through grpc.CallOptions). +// +// Retry logic is available *only for ServerStreams*, i.e. 1:n streams, as the internal logic needs +// to buffer the messages sent by the client. If retry is enabled on any other streams (ClientStreams, +// BidiStreams), the retry interceptor will fail the call. +func StreamClientInterceptor(optFuncs ...CallOption) grpc.StreamClientInterceptor { + intOpts := reuseOrNewWithCallOptions(defaultOptions, optFuncs) + return func(parentCtx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) { + grpcOpts, retryOpts := filterCallOptions(opts) + callOpts := reuseOrNewWithCallOptions(intOpts, retryOpts) + // short circuit for simplicity, and avoiding allocations. + if callOpts.max == 0 { + return streamer(parentCtx, desc, cc, method, grpcOpts...) + } + if desc.ClientStreams { + return nil, status.Errorf(codes.Unimplemented, "grpc_retry: cannot retry on ClientStreams, set grpc_retry.Disable()") + } + + var lastErr error + for attempt := uint(0); attempt < callOpts.max; attempt++ { + if err := waitRetryBackoff(attempt, parentCtx, callOpts); err != nil { + return nil, err + } + callCtx := perCallContext(parentCtx, callOpts, 0) + + var newStreamer grpc.ClientStream + newStreamer, lastErr = streamer(callCtx, desc, cc, method, grpcOpts...) + if lastErr == nil { + retryingStreamer := &serverStreamingRetryingStream{ + ClientStream: newStreamer, + callOpts: callOpts, + parentCtx: parentCtx, + streamerCall: func(ctx context.Context) (grpc.ClientStream, error) { + return streamer(ctx, desc, cc, method, grpcOpts...) + }, + } + return retryingStreamer, nil + } + + logTrace(parentCtx, "grpc_retry attempt: %d, got err: %v", attempt, lastErr) + if isContextError(lastErr) { + if parentCtx.Err() != nil { + logTrace(parentCtx, "grpc_retry attempt: %d, parent context error: %v", attempt, parentCtx.Err()) + // its the parent context deadline or cancellation. + return nil, lastErr + } else if callOpts.perCallTimeout != 0 { + // We have set a perCallTimeout in the retry middleware, which would result in a context error if + // the deadline was exceeded, in which case try again. + logTrace(parentCtx, "grpc_retry attempt: %d, context error from retry call", attempt) + continue + } + } + if !isRetriable(lastErr, callOpts) { + return nil, lastErr + } + } + return nil, lastErr + } +} + +// type serverStreamingRetryingStream is the implementation of grpc.ClientStream that acts as a +// proxy to the underlying call. If any of the RecvMsg() calls fail, it will try to reestablish +// a new ClientStream according to the retry policy. +type serverStreamingRetryingStream struct { + grpc.ClientStream + bufferedSends []interface{} // single message that the client can sen + wasClosedSend bool // indicates that CloseSend was closed + parentCtx context.Context + callOpts *options + streamerCall func(ctx context.Context) (grpc.ClientStream, error) + mu sync.RWMutex +} + +func (s *serverStreamingRetryingStream) setStream(clientStream grpc.ClientStream) { + s.mu.Lock() + s.ClientStream = clientStream + s.mu.Unlock() +} + +func (s *serverStreamingRetryingStream) getStream() grpc.ClientStream { + s.mu.RLock() + defer s.mu.RUnlock() + return s.ClientStream +} + +func (s *serverStreamingRetryingStream) SendMsg(m interface{}) error { + s.mu.Lock() + s.bufferedSends = append(s.bufferedSends, m) + s.mu.Unlock() + return s.getStream().SendMsg(m) +} + +func (s *serverStreamingRetryingStream) CloseSend() error { + s.mu.Lock() + s.wasClosedSend = true + s.mu.Unlock() + return s.getStream().CloseSend() +} + +func (s *serverStreamingRetryingStream) Header() (metadata.MD, error) { + return s.getStream().Header() +} + +func (s *serverStreamingRetryingStream) Trailer() metadata.MD { + return s.getStream().Trailer() +} + +func (s *serverStreamingRetryingStream) RecvMsg(m interface{}) error { + attemptRetry, lastErr := s.receiveMsgAndIndicateRetry(m) + if !attemptRetry { + return lastErr // success or hard failure + } + // We start off from attempt 1, because zeroth was already made on normal SendMsg(). + for attempt := uint(1); attempt < s.callOpts.max; attempt++ { + if err := waitRetryBackoff(attempt, s.parentCtx, s.callOpts); err != nil { + return err + } + callCtx := perCallContext(s.parentCtx, s.callOpts, attempt) + newStream, err := s.reestablishStreamAndResendBuffer(callCtx) + if err != nil { + // Retry dial and transport errors of establishing stream as grpc doesn't retry. + if isRetriable(err, s.callOpts) { + continue + } + return err + } + + s.setStream(newStream) + attemptRetry, lastErr = s.receiveMsgAndIndicateRetry(m) + //fmt.Printf("Received message and indicate: %v %v\n", attemptRetry, lastErr) + if !attemptRetry { + return lastErr + } + } + return lastErr +} + +func (s *serverStreamingRetryingStream) receiveMsgAndIndicateRetry(m interface{}) (bool, error) { + err := s.getStream().RecvMsg(m) + if err == nil || err == io.EOF { + return false, err + } + if isContextError(err) { + if s.parentCtx.Err() != nil { + logTrace(s.parentCtx, "grpc_retry parent context error: %v", s.parentCtx.Err()) + return false, err + } else if s.callOpts.perCallTimeout != 0 { + // We have set a perCallTimeout in the retry middleware, which would result in a context error if + // the deadline was exceeded, in which case try again. + logTrace(s.parentCtx, "grpc_retry context error from retry call") + return true, err + } + } + return isRetriable(err, s.callOpts), err +} + +func (s *serverStreamingRetryingStream) reestablishStreamAndResendBuffer( + callCtx context.Context, +) (grpc.ClientStream, error) { + s.mu.RLock() + bufferedSends := s.bufferedSends + s.mu.RUnlock() + newStream, err := s.streamerCall(callCtx) + if err != nil { + logTrace(callCtx, "grpc_retry failed redialing new stream: %v", err) + return nil, err + } + for _, msg := range bufferedSends { + if err := newStream.SendMsg(msg); err != nil { + logTrace(callCtx, "grpc_retry failed resending message: %v", err) + return nil, err + } + } + if err := newStream.CloseSend(); err != nil { + logTrace(callCtx, "grpc_retry failed CloseSend on new stream %v", err) + return nil, err + } + return newStream, nil +} + +func waitRetryBackoff(attempt uint, parentCtx context.Context, callOpts *options) error { + var waitTime time.Duration = 0 + if attempt > 0 { + waitTime = callOpts.backoffFunc(parentCtx, attempt) + } + if waitTime > 0 { + logTrace(parentCtx, "grpc_retry attempt: %d, backoff for %v", attempt, waitTime) + timer := time.NewTimer(waitTime) + select { + case <-parentCtx.Done(): + timer.Stop() + return contextErrToGrpcErr(parentCtx.Err()) + case <-timer.C: + } + } + return nil +} + +func isRetriable(err error, callOpts *options) bool { + errCode := status.Code(err) + if isContextError(err) { + // context errors are not retriable based on user settings. + return false + } + for _, code := range callOpts.codes { + if code == errCode { + return true + } + } + return false +} + +func isContextError(err error) bool { + code := status.Code(err) + return code == codes.DeadlineExceeded || code == codes.Canceled +} + +func perCallContext(parentCtx context.Context, callOpts *options, attempt uint) context.Context { + ctx := parentCtx + if callOpts.perCallTimeout != 0 { + ctx, _ = context.WithTimeout(ctx, callOpts.perCallTimeout) + } + if attempt > 0 && callOpts.includeHeader { + mdClone := metautils.ExtractOutgoing(ctx).Clone().Set(AttemptMetadataKey, strconv.FormatUint(uint64(attempt), 10)) + ctx = mdClone.ToOutgoing(ctx) + } + return ctx +} + +func contextErrToGrpcErr(err error) error { + switch err { + case context.DeadlineExceeded: + return status.Error(codes.DeadlineExceeded, err.Error()) + case context.Canceled: + return status.Error(codes.Canceled, err.Error()) + default: + return status.Error(codes.Unknown, err.Error()) + } +} + +func logTrace(ctx context.Context, format string, a ...interface{}) { + tr, ok := trace.FromContext(ctx) + if !ok { + return + } + tr.LazyPrintf(format, a...) +} diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils/backoff.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils/backoff.go new file mode 100644 index 00000000000..4e69a6305aa --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils/backoff.go @@ -0,0 +1,28 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +/* +Backoff Helper Utilities + +Implements common backoff features. +*/ +package backoffutils + +import ( + "math/rand" + "time" +) + +// JitterUp adds random jitter to the duration. +// +// This adds or subtracts time from the duration within a given jitter fraction. +// For example for 10s and jitter 0.1, it will return a time within [9s, 11s]) +func JitterUp(duration time.Duration, jitter float64) time.Duration { + multiplier := jitter * (rand.Float64()*2 - 1) + return time.Duration(float64(duration) * (1 + multiplier)) +} + +// ExponentBase2 computes 2^(a-1) where a >= 1. If a is 0, the result is 0. +func ExponentBase2(a uint) uint { + return (1 << a) >> 1 +} diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/doc.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/doc.go new file mode 100644 index 00000000000..1ed9bb499b3 --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/doc.go @@ -0,0 +1,19 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +/* +Package `metautils` provides convenience functions for dealing with gRPC metadata.MD objects inside +Context handlers. + +While the upstream grpc-go package contains decent functionality (see https://github.com/grpc/grpc-go/blob/master/Documentation/grpc-metadata.md) +they are hard to use. + +The majority of functions center around the NiceMD, which is a convenience wrapper around metadata.MD. For example +the following code allows you to easily extract incoming metadata (server handler) and put it into a new client context +metadata. + + nmd := metautils.ExtractIncoming(serverCtx).Clone(":authorization", ":custom") + clientCtx := nmd.Set("x-client-header", "2").Set("x-another", "3").ToOutgoing(ctx) +*/ + +package metautils diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/nicemd.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/nicemd.go new file mode 100644 index 00000000000..15225d710ac --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/nicemd.go @@ -0,0 +1,126 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +package metautils + +import ( + "context" + "strings" + + "google.golang.org/grpc/metadata" +) + +// NiceMD is a convenience wrapper defining extra functions on the metadata. +type NiceMD metadata.MD + +// ExtractIncoming extracts an inbound metadata from the server-side context. +// +// This function always returns a NiceMD wrapper of the metadata.MD, in case the context doesn't have metadata it returns +// a new empty NiceMD. +func ExtractIncoming(ctx context.Context) NiceMD { + md, ok := metadata.FromIncomingContext(ctx) + if !ok { + return NiceMD(metadata.Pairs()) + } + return NiceMD(md) +} + +// ExtractOutgoing extracts an outbound metadata from the client-side context. +// +// This function always returns a NiceMD wrapper of the metadata.MD, in case the context doesn't have metadata it returns +// a new empty NiceMD. +func ExtractOutgoing(ctx context.Context) NiceMD { + md, ok := metadata.FromOutgoingContext(ctx) + if !ok { + return NiceMD(metadata.Pairs()) + } + return NiceMD(md) +} + +// Clone performs a *deep* copy of the metadata.MD. +// +// You can specify the lower-case copiedKeys to only copy certain allow-listed keys. If no keys are explicitly allow-listed +// all keys get copied. +func (m NiceMD) Clone(copiedKeys ...string) NiceMD { + newMd := NiceMD(metadata.Pairs()) + for k, vv := range m { + found := false + if len(copiedKeys) == 0 { + found = true + } else { + for _, allowedKey := range copiedKeys { + if strings.EqualFold(allowedKey, k) { + found = true + break + } + } + } + if !found { + continue + } + newMd[k] = make([]string, len(vv)) + copy(newMd[k], vv) + } + return newMd +} + +// ToOutgoing sets the given NiceMD as a client-side context for dispatching. +func (m NiceMD) ToOutgoing(ctx context.Context) context.Context { + return metadata.NewOutgoingContext(ctx, metadata.MD(m)) +} + +// ToIncoming sets the given NiceMD as a server-side context for dispatching. +// +// This is mostly useful in ServerInterceptors.. +func (m NiceMD) ToIncoming(ctx context.Context) context.Context { + return metadata.NewIncomingContext(ctx, metadata.MD(m)) +} + +// Get retrieves a single value from the metadata. +// +// It works analogously to http.Header.Get, returning the first value if there are many set. If the value is not set, +// an empty string is returned. +// +// The function is binary-key safe. +func (m NiceMD) Get(key string) string { + k := strings.ToLower(key) + vv, ok := m[k] + if !ok { + return "" + } + return vv[0] +} + +// Del retrieves a single value from the metadata. +// +// It works analogously to http.Header.Del, deleting all values if they exist. +// +// The function is binary-key safe. + +func (m NiceMD) Del(key string) NiceMD { + k := strings.ToLower(key) + delete(m, k) + return m +} + +// Set sets the given value in a metadata. +// +// It works analogously to http.Header.Set, overwriting all previous metadata values. +// +// The function is binary-key safe. +func (m NiceMD) Set(key string, value string) NiceMD { + k := strings.ToLower(key) + m[k] = []string{value} + return m +} + +// Add retrieves a single value from the metadata. +// +// It works analogously to http.Header.Add, as it appends to any existing values associated with key. +// +// The function is binary-key safe. +func (m NiceMD) Add(key string, value string) NiceMD { + k := strings.ToLower(key) + m[k] = append(m[k], value) + return m +} diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/BUILD.bazel b/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/BUILD.bazel index 78d7c9f5c88..a65d88eb865 100644 --- a/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/BUILD.bazel +++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/BUILD.bazel @@ -73,7 +73,7 @@ go_test( "@org_golang_google_genproto_googleapis_api//httpbody", "@org_golang_google_genproto_googleapis_rpc//errdetails", "@org_golang_google_genproto_googleapis_rpc//status", - "@org_golang_google_grpc//:go_default_library", + "@org_golang_google_grpc//:grpc", "@org_golang_google_grpc//codes", "@org_golang_google_grpc//health/grpc_health_v1", "@org_golang_google_grpc//metadata", diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/context.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/context.go index 5dd4e447862..2f2b342431d 100644 --- a/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/context.go +++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/context.go @@ -49,6 +49,7 @@ var malformedHTTPHeaders = map[string]struct{}{ type ( rpcMethodKey struct{} httpPathPatternKey struct{} + httpPatternKey struct{} AnnotateContextOption func(ctx context.Context) context.Context ) @@ -404,3 +405,13 @@ func HTTPPathPattern(ctx context.Context) (string, bool) { func withHTTPPathPattern(ctx context.Context, httpPathPattern string) context.Context { return context.WithValue(ctx, httpPathPatternKey{}, httpPathPattern) } + +// HTTPPattern returns the HTTP path pattern struct relating to the HTTP handler, if one exists. +func HTTPPattern(ctx context.Context) (Pattern, bool) { + v, ok := ctx.Value(httpPatternKey{}).(Pattern) + return v, ok +} + +func withHTTPPattern(ctx context.Context, httpPattern Pattern) context.Context { + return context.WithValue(ctx, httpPatternKey{}, httpPattern) +} diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/errors.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/errors.go index 5682998699a..01f57341918 100644 --- a/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/errors.go +++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/errors.go @@ -93,6 +93,7 @@ func HTTPError(ctx context.Context, mux *ServeMux, marshaler Marshaler, w http.R func DefaultHTTPErrorHandler(ctx context.Context, mux *ServeMux, marshaler Marshaler, w http.ResponseWriter, r *http.Request, err error) { // return Internal when Marshal failed const fallback = `{"code": 13, "message": "failed to marshal error message"}` + const fallbackRewriter = `{"code": 13, "message": "failed to rewrite error message"}` var customStatus *HTTPStatusError if errors.As(err, &customStatus) { @@ -100,19 +101,28 @@ func DefaultHTTPErrorHandler(ctx context.Context, mux *ServeMux, marshaler Marsh } s := status.Convert(err) - pb := s.Proto() w.Header().Del("Trailer") w.Header().Del("Transfer-Encoding") - contentType := marshaler.ContentType(pb) + respRw, err := mux.forwardResponseRewriter(ctx, s.Proto()) + if err != nil { + grpclog.Errorf("Failed to rewrite error message %q: %v", s, err) + w.WriteHeader(http.StatusInternalServerError) + if _, err := io.WriteString(w, fallbackRewriter); err != nil { + grpclog.Errorf("Failed to write response: %v", err) + } + return + } + + contentType := marshaler.ContentType(respRw) w.Header().Set("Content-Type", contentType) if s.Code() == codes.Unauthenticated { w.Header().Set("WWW-Authenticate", s.Message()) } - buf, merr := marshaler.Marshal(pb) + buf, merr := marshaler.Marshal(respRw) if merr != nil { grpclog.Errorf("Failed to marshal error message %q: %v", s, merr) w.WriteHeader(http.StatusInternalServerError) diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/handler.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/handler.go index de1eef1f4f8..9f50a569e9b 100644 --- a/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/handler.go +++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/handler.go @@ -3,6 +3,7 @@ package runtime import ( "context" "errors" + "fmt" "io" "net/http" "net/textproto" @@ -55,20 +56,27 @@ func ForwardResponseStream(ctx context.Context, mux *ServeMux, marshaler Marshal return } + respRw, err := mux.forwardResponseRewriter(ctx, resp) + if err != nil { + grpclog.Errorf("Rewrite error: %v", err) + handleForwardResponseStreamError(ctx, wroteHeader, marshaler, w, req, mux, err, delimiter) + return + } + if !wroteHeader { - w.Header().Set("Content-Type", marshaler.ContentType(resp)) + w.Header().Set("Content-Type", marshaler.ContentType(respRw)) } var buf []byte - httpBody, isHTTPBody := resp.(*httpbody.HttpBody) + httpBody, isHTTPBody := respRw.(*httpbody.HttpBody) switch { - case resp == nil: + case respRw == nil: buf, err = marshaler.Marshal(errorChunk(status.New(codes.Internal, "empty response"))) case isHTTPBody: buf = httpBody.GetData() default: - result := map[string]interface{}{"result": resp} - if rb, ok := resp.(responseBody); ok { + result := map[string]interface{}{"result": respRw} + if rb, ok := respRw.(responseBody); ok { result["result"] = rb.XXX_ResponseBody() } @@ -164,12 +172,17 @@ func ForwardResponseMessage(ctx context.Context, mux *ServeMux, marshaler Marsha HTTPError(ctx, mux, marshaler, w, req, err) return } + respRw, err := mux.forwardResponseRewriter(ctx, resp) + if err != nil { + grpclog.Errorf("Rewrite error: %v", err) + HTTPError(ctx, mux, marshaler, w, req, err) + return + } var buf []byte - var err error - if rb, ok := resp.(responseBody); ok { + if rb, ok := respRw.(responseBody); ok { buf, err = marshaler.Marshal(rb.XXX_ResponseBody()) } else { - buf, err = marshaler.Marshal(resp) + buf, err = marshaler.Marshal(respRw) } if err != nil { grpclog.Errorf("Marshal error: %v", err) @@ -201,8 +214,7 @@ func handleForwardResponseOptions(ctx context.Context, w http.ResponseWriter, re } for _, opt := range opts { if err := opt(ctx, w, resp); err != nil { - grpclog.Errorf("Error handling ForwardResponseOptions: %v", err) - return err + return fmt.Errorf("error handling ForwardResponseOptions: %w", err) } } return nil diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/mux.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/mux.go index ed9a7e4387d..60c2065ddcb 100644 --- a/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/mux.go +++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/v2/runtime/mux.go @@ -48,12 +48,19 @@ var encodedPathSplitter = regexp.MustCompile("(/|%2F)") // A HandlerFunc handles a specific pair of path pattern and HTTP method. type HandlerFunc func(w http.ResponseWriter, r *http.Request, pathParams map[string]string) +// A Middleware handler wraps another HandlerFunc to do some pre- and/or post-processing of the request. This is used as an alternative to gRPC interceptors when using the direct-to-implementation +// registration methods. It is generally recommended to use gRPC client or server interceptors instead +// where possible. +type Middleware func(HandlerFunc) HandlerFunc + // ServeMux is a request multiplexer for grpc-gateway. // It matches http requests to patterns and invokes the corresponding handler. type ServeMux struct { // handlers maps HTTP method to a list of handlers. handlers map[string][]handler + middlewares []Middleware forwardResponseOptions []func(context.Context, http.ResponseWriter, proto.Message) error + forwardResponseRewriter ForwardResponseRewriter marshalers marshalerRegistry incomingHeaderMatcher HeaderMatcherFunc outgoingHeaderMatcher HeaderMatcherFunc @@ -69,6 +76,24 @@ type ServeMux struct { // ServeMuxOption is an option that can be given to a ServeMux on construction. type ServeMuxOption func(*ServeMux) +// ForwardResponseRewriter is the signature of a function that is capable of rewriting messages +// before they are forwarded in a unary, stream, or error response. +type ForwardResponseRewriter func(ctx context.Context, response proto.Message) (any, error) + +// WithForwardResponseRewriter returns a ServeMuxOption that allows for implementers to insert logic +// that can rewrite the final response before it is forwarded. +// +// The response rewriter function is called during unary message forwarding, stream message +// forwarding and when errors are being forwarded. +// +// NOTE: Using this option will likely make what is generated by `protoc-gen-openapiv2` incorrect. +// Since this option involves making runtime changes to the response shape or type. +func WithForwardResponseRewriter(fwdResponseRewriter ForwardResponseRewriter) ServeMuxOption { + return func(sm *ServeMux) { + sm.forwardResponseRewriter = fwdResponseRewriter + } +} + // WithForwardResponseOption returns a ServeMuxOption representing the forwardResponseOption. // // forwardResponseOption is an option that will be called on the relevant context.Context, @@ -89,6 +114,15 @@ func WithUnescapingMode(mode UnescapingMode) ServeMuxOption { } } +// WithMiddlewares sets server middleware for all handlers. This is useful as an alternative to gRPC +// interceptors when using the direct-to-implementation registration methods and cannot rely +// on gRPC interceptors. It's recommended to use gRPC interceptors instead if possible. +func WithMiddlewares(middlewares ...Middleware) ServeMuxOption { + return func(serveMux *ServeMux) { + serveMux.middlewares = append(serveMux.middlewares, middlewares...) + } +} + // SetQueryParameterParser sets the query parameter parser, used to populate message from query parameters. // Configuring this will mean the generated OpenAPI output is no longer correct, and it should be // done with careful consideration. @@ -277,13 +311,14 @@ func WithHealthzEndpoint(healthCheckClient grpc_health_v1.HealthClient) ServeMux // NewServeMux returns a new ServeMux whose internal mapping is empty. func NewServeMux(opts ...ServeMuxOption) *ServeMux { serveMux := &ServeMux{ - handlers: make(map[string][]handler), - forwardResponseOptions: make([]func(context.Context, http.ResponseWriter, proto.Message) error, 0), - marshalers: makeMarshalerMIMERegistry(), - errorHandler: DefaultHTTPErrorHandler, - streamErrorHandler: DefaultStreamErrorHandler, - routingErrorHandler: DefaultRoutingErrorHandler, - unescapingMode: UnescapingModeDefault, + handlers: make(map[string][]handler), + forwardResponseOptions: make([]func(context.Context, http.ResponseWriter, proto.Message) error, 0), + forwardResponseRewriter: func(ctx context.Context, response proto.Message) (any, error) { return response, nil }, + marshalers: makeMarshalerMIMERegistry(), + errorHandler: DefaultHTTPErrorHandler, + streamErrorHandler: DefaultStreamErrorHandler, + routingErrorHandler: DefaultRoutingErrorHandler, + unescapingMode: UnescapingModeDefault, } for _, opt := range opts { @@ -305,6 +340,9 @@ func NewServeMux(opts ...ServeMuxOption) *ServeMux { // Handle associates "h" to the pair of HTTP method and path pattern. func (s *ServeMux) Handle(meth string, pat Pattern, h HandlerFunc) { + if len(s.middlewares) > 0 { + h = chainMiddlewares(s.middlewares)(h) + } s.handlers[meth] = append([]handler{{pat: pat, h: h}}, s.handlers[meth]...) } @@ -405,7 +443,7 @@ func (s *ServeMux) ServeHTTP(w http.ResponseWriter, r *http.Request) { } continue } - h.h(w, r, pathParams) + s.handleHandler(h, w, r, pathParams) return } @@ -458,7 +496,7 @@ func (s *ServeMux) ServeHTTP(w http.ResponseWriter, r *http.Request) { s.errorHandler(ctx, s, outboundMarshaler, w, r, sterr) return } - h.h(w, r, pathParams) + s.handleHandler(h, w, r, pathParams) return } _, outboundMarshaler := MarshalerForRequest(s, r) @@ -484,3 +522,16 @@ type handler struct { pat Pattern h HandlerFunc } + +func (s *ServeMux) handleHandler(h handler, w http.ResponseWriter, r *http.Request, pathParams map[string]string) { + h.h(w, r.WithContext(withHTTPPattern(r.Context(), h.pat)), pathParams) +} + +func chainMiddlewares(mws []Middleware) Middleware { + return func(next HandlerFunc) HandlerFunc { + for i := len(mws); i > 0; i-- { + next = mws[i-1](next) + } + return next + } +} diff --git a/vendor/github.com/nexus-rpc/sdk-go/LICENSE b/vendor/github.com/nexus-rpc/sdk-go/LICENSE new file mode 100644 index 00000000000..1a8b19536b2 --- /dev/null +++ b/vendor/github.com/nexus-rpc/sdk-go/LICENSE @@ -0,0 +1,21 @@ +MIT License + +Copyright (c) 2023 Temporal Technologies Inc. All Rights Reserved + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/vendor/github.com/nexus-rpc/sdk-go/nexus/api.go b/vendor/github.com/nexus-rpc/sdk-go/nexus/api.go new file mode 100644 index 00000000000..db3eaf4bd20 --- /dev/null +++ b/vendor/github.com/nexus-rpc/sdk-go/nexus/api.go @@ -0,0 +1,335 @@ +// Package nexus provides client and server implementations of the Nexus [HTTP API] +// +// [HTTP API]: https://github.com/nexus-rpc/api +package nexus + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "mime" + "net/http" + "net/url" + "strings" + "time" +) + +// Package version. +const version = "v0.0.9" + +const ( + // Nexus specific headers. + headerOperationState = "Nexus-Operation-State" + headerOperationID = "Nexus-Operation-Id" + headerRequestID = "Nexus-Request-Id" + headerLink = "Nexus-Link" + + // HeaderRequestTimeout is the total time to complete a Nexus HTTP request. + HeaderRequestTimeout = "Request-Timeout" +) + +const contentTypeJSON = "application/json" + +// Query param for passing a callback URL. +const ( + queryCallbackURL = "callback" + // Query param for passing wait duration. + queryWait = "wait" +) + +const ( + statusOperationRunning = http.StatusPreconditionFailed + // HTTP status code for failed operation responses. + statusOperationFailed = http.StatusFailedDependency + StatusDownstreamTimeout = 520 +) + +// A Failure represents failed handler invocations as well as `failed` or `canceled` operation results. +type Failure struct { + // A simple text message. + Message string `json:"message"` + // A key-value mapping for additional context. Useful for decoding the 'details' field, if needed. + Metadata map[string]string `json:"metadata,omitempty"` + // Additional JSON serializable structured data. + Details json.RawMessage `json:"details,omitempty"` +} + +// UnsuccessfulOperationError represents "failed" and "canceled" operation results. +type UnsuccessfulOperationError struct { + State OperationState + Failure Failure +} + +// Error implements the error interface. +func (e *UnsuccessfulOperationError) Error() string { + if e.Failure.Message != "" { + return fmt.Sprintf("operation %s: %s", e.State, e.Failure.Message) + } + return fmt.Sprintf("operation %s", e.State) +} + +// ErrOperationStillRunning indicates that an operation is still running while trying to get its result. +var ErrOperationStillRunning = errors.New("operation still running") + +// OperationInfo conveys information about an operation. +type OperationInfo struct { + // ID of the operation. + ID string `json:"id"` + // State of the operation. + State OperationState `json:"state"` +} + +// OperationState represents the variable states of an operation. +type OperationState string + +const ( + // "running" operation state. Indicates an operation is started and not yet completed. + OperationStateRunning OperationState = "running" + // "succeeded" operation state. Indicates an operation completed successfully. + OperationStateSucceeded OperationState = "succeeded" + // "failed" operation state. Indicates an operation completed as failed. + OperationStateFailed OperationState = "failed" + // "canceled" operation state. Indicates an operation completed as canceled. + OperationStateCanceled OperationState = "canceled" +) + +// isMediaTypeJSON returns true if the given content type's media type is application/json. +func isMediaTypeJSON(contentType string) bool { + if contentType == "" { + return false + } + mediaType, _, err := mime.ParseMediaType(contentType) + return err == nil && mediaType == "application/json" +} + +// isMediaTypeOctetStream returns true if the given content type's media type is application/octet-stream. +func isMediaTypeOctetStream(contentType string) bool { + if contentType == "" { + return false + } + mediaType, _, err := mime.ParseMediaType(contentType) + return err == nil && mediaType == "application/octet-stream" +} + +// Header is a mapping of string to string. +// It is used throughout the framework to transmit metadata. +type Header map[string]string + +// Get is a case-insensitive key lookup from the header map. +func (h Header) Get(k string) string { + return h[strings.ToLower(k)] +} + +func prefixStrippedHTTPHeaderToNexusHeader(httpHeader http.Header, prefix string) Header { + header := Header{} + for k, v := range httpHeader { + lowerK := strings.ToLower(k) + if strings.HasPrefix(lowerK, prefix) { + // Nexus headers can only have single values, ignore multiple values. + header[lowerK[len(prefix):]] = v[0] + } + } + return header +} + +func addContentHeaderToHTTPHeader(nexusHeader Header, httpHeader http.Header) http.Header { + for k, v := range nexusHeader { + httpHeader.Set("Content-"+k, v) + } + return httpHeader +} + +func addCallbackHeaderToHTTPHeader(nexusHeader Header, httpHeader http.Header) http.Header { + for k, v := range nexusHeader { + httpHeader.Set("Nexus-Callback-"+k, v) + } + return httpHeader +} + +func addLinksToHTTPHeader(links []Link, httpHeader http.Header) error { + for _, link := range links { + encodedLink, err := encodeLink(link) + if err != nil { + return err + } + httpHeader.Add(headerLink, encodedLink) + } + return nil +} + +func getLinksFromHeader(httpHeader http.Header) ([]Link, error) { + var links []Link + headerValues := httpHeader.Values(headerLink) + if len(headerValues) == 0 { + return nil, nil + } + for _, encodedLink := range strings.Split(strings.Join(headerValues, ","), ",") { + link, err := decodeLink(encodedLink) + if err != nil { + return nil, err + } + links = append(links, link) + } + return links, nil +} + +func httpHeaderToNexusHeader(httpHeader http.Header, excludePrefixes ...string) Header { + header := Header{} +headerLoop: + for k, v := range httpHeader { + lowerK := strings.ToLower(k) + for _, prefix := range excludePrefixes { + if strings.HasPrefix(lowerK, prefix) { + continue headerLoop + } + } + // Nexus headers can only have single values, ignore multiple values. + header[lowerK] = v[0] + } + return header +} + +func addNexusHeaderToHTTPHeader(nexusHeader Header, httpHeader http.Header) http.Header { + for k, v := range nexusHeader { + httpHeader.Set(k, v) + } + return httpHeader +} + +func addContextTimeoutToHTTPHeader(ctx context.Context, httpHeader http.Header) http.Header { + deadline, ok := ctx.Deadline() + if !ok { + return httpHeader + } + httpHeader.Set(HeaderRequestTimeout, time.Until(deadline).String()) + return httpHeader +} + +// Link contains an URL and a Type that can be used to decode the URL. +// Links can contain any arbitrary information as a percent-encoded URL. +// It can be used to pass information about the caller to the handler, or vice-versa. +type Link struct { + // URL information about the link. + // It must be URL percent-encoded. + URL *url.URL + // Type can describe an actual data type for decoding the URL. + // Valid chars: alphanumeric, '_', '.', '/' + Type string +} + +const linkTypeKey = "type" + +// decodeLink encodes the link to Nexus-Link header value. +// It follows the same format of HTTP Link header: https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Link +func encodeLink(link Link) (string, error) { + if err := validateLinkURL(link.URL); err != nil { + return "", fmt.Errorf("failed to encode link: %w", err) + } + if err := validateLinkType(link.Type); err != nil { + return "", fmt.Errorf("failed to encode link: %w", err) + } + return fmt.Sprintf(`<%s>; %s="%s"`, link.URL.String(), linkTypeKey, link.Type), nil +} + +// decodeLink decodes the Nexus-Link header values. +// It must have the same format of HTTP Link header: https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Link +func decodeLink(encodedLink string) (Link, error) { + var link Link + encodedLink = strings.TrimSpace(encodedLink) + if len(encodedLink) == 0 { + return link, fmt.Errorf("failed to parse link header: value is empty") + } + + if encodedLink[0] != '<' { + return link, fmt.Errorf("failed to parse link header: invalid format: %s", encodedLink) + } + urlEnd := strings.Index(encodedLink, ">") + if urlEnd == -1 { + return link, fmt.Errorf("failed to parse link header: invalid format: %s", encodedLink) + } + urlStr := strings.TrimSpace(encodedLink[1:urlEnd]) + if len(urlStr) == 0 { + return link, fmt.Errorf("failed to parse link header: url is empty") + } + u, err := url.Parse(urlStr) + if err != nil { + return link, fmt.Errorf("failed to parse link header: invalid url: %s", urlStr) + } + if err := validateLinkURL(u); err != nil { + return link, fmt.Errorf("failed to parse link header: %w", err) + } + link.URL = u + + params := strings.Split(encodedLink[urlEnd+1:], ";") + // must contain at least one semi-colon, and first param must be empty since + // it corresponds to the url part parsed above. + if len(params) < 2 { + return link, fmt.Errorf("failed to parse link header: invalid format: %s", encodedLink) + } + if strings.TrimSpace(params[0]) != "" { + return link, fmt.Errorf("failed to parse link header: invalid format: %s", encodedLink) + } + + typeKeyFound := false + for _, param := range params[1:] { + param = strings.TrimSpace(param) + if len(param) == 0 { + return link, fmt.Errorf("failed to parse link header: parameter is empty: %s", encodedLink) + } + kv := strings.SplitN(param, "=", 2) + if len(kv) != 2 { + return link, fmt.Errorf("failed to parse link header: invalid parameter format: %s", param) + } + key := strings.TrimSpace(kv[0]) + val := strings.TrimSpace(kv[1]) + if strings.HasPrefix(val, `"`) != strings.HasSuffix(val, `"`) { + return link, fmt.Errorf( + "failed to parse link header: parameter value missing double-quote: %s", + param, + ) + } + if strings.HasPrefix(val, `"`) { + val = val[1 : len(val)-1] + } + if key == linkTypeKey { + if err := validateLinkType(val); err != nil { + return link, fmt.Errorf("failed to parse link header: %w", err) + } + link.Type = val + typeKeyFound = true + } + } + if !typeKeyFound { + return link, fmt.Errorf( + "failed to parse link header: %q key not found: %s", + linkTypeKey, + encodedLink, + ) + } + + return link, nil +} + +func validateLinkURL(value *url.URL) error { + if value == nil || value.String() == "" { + return fmt.Errorf("url is empty") + } + _, err := url.ParseQuery(value.RawQuery) + if err != nil { + return fmt.Errorf("url query not percent-encoded: %s", value) + } + return nil +} + +func validateLinkType(value string) error { + if len(value) == 0 { + return fmt.Errorf("link type is empty") + } + for _, c := range value { + if !(c >= 'a' && c <= 'z') && !(c >= 'A' && c <= 'Z') && !(c >= '0' && c <= '9') && c != '_' && c != '.' && c != '/' { + return fmt.Errorf("link type contains invalid char (valid chars: alphanumeric, '_', '.', '/')") + } + } + return nil +} diff --git a/vendor/github.com/nexus-rpc/sdk-go/nexus/client.go b/vendor/github.com/nexus-rpc/sdk-go/nexus/client.go new file mode 100644 index 00000000000..18f7778c5a7 --- /dev/null +++ b/vendor/github.com/nexus-rpc/sdk-go/nexus/client.go @@ -0,0 +1,413 @@ +package nexus + +import ( + "bytes" + "context" + "encoding/json" + "errors" + "fmt" + "io" + "maps" + "math" + "net/http" + "net/url" + "strconv" + "time" + + "github.com/google/uuid" +) + +// ClientOptions are options for creating a Client. +type ClientOptions struct { + // Base URL for all requests. Required. + BaseURL string + // Service name. Required. + Service string + // A function for making HTTP requests. + // Defaults to [http.DefaultClient.Do]. + HTTPCaller func(*http.Request) (*http.Response, error) + // A [Serializer] to customize client serialization behavior. + // By default the client handles, JSONables, byte slices, and nil. + Serializer Serializer +} + +// User-Agent header set on HTTP requests. +const userAgent = "Nexus-go-sdk/" + version + +const headerUserAgent = "User-Agent" + +var errEmptyOperationName = errors.New("empty operation name") + +var errEmptyOperationID = errors.New("empty operation ID") + +var errOperationWaitTimeout = errors.New("operation wait timeout") + +// Error that indicates a client encountered something unexpected in the server's response. +type UnexpectedResponseError struct { + // Error message. + Message string + // The HTTP response. The response body will have already been read into memory and does not need to be closed. + Response *http.Response + // Optional failure that may have been emedded in the HTTP response body. + Failure *Failure +} + +// Error implements the error interface. +func (e *UnexpectedResponseError) Error() string { + return e.Message +} + +func newUnexpectedResponseError(message string, response *http.Response, body []byte) error { + var failure *Failure + if isMediaTypeJSON(response.Header.Get("Content-Type")) { + if err := json.Unmarshal(body, &failure); err == nil && failure.Message != "" { + message += ": " + failure.Message + } + } + + return &UnexpectedResponseError{ + Message: message, + Response: response, + Failure: failure, + } +} + +// A Client makes Nexus service requests as defined in the [Nexus HTTP API]. +// +// It can start a new operation and get an [OperationHandle] to an existing, asynchronous operation. +// +// Use an [OperationHandle] to cancel, get the result of, and get information about asynchronous operations. +// +// OperationHandles can be obtained either by starting new operations or by calling [Client.NewHandle] for existing +// operations. +// +// [Nexus HTTP API]: https://github.com/nexus-rpc/api +type Client struct { + // The options this client was created with after applying defaults. + options ClientOptions + serviceBaseURL *url.URL +} + +// NewClient creates a new [Client] from provided [ClientOptions]. +// BaseURL and Service are required. +func NewClient(options ClientOptions) (*Client, error) { + if options.HTTPCaller == nil { + options.HTTPCaller = http.DefaultClient.Do + } + if options.BaseURL == "" { + return nil, errors.New("empty BaseURL") + } + if options.Service == "" { + return nil, errors.New("empty Service") + } + var baseURL *url.URL + var err error + baseURL, err = url.Parse(options.BaseURL) + if err != nil { + return nil, err + } + if baseURL.Scheme != "http" && baseURL.Scheme != "https" { + return nil, fmt.Errorf("invalid URL scheme: %s", baseURL.Scheme) + } + if options.Serializer == nil { + options.Serializer = defaultSerializer + } + + return &Client{ + options: options, + serviceBaseURL: baseURL, + }, nil +} + +// ClientStartOperationResult is the return type of [Client.StartOperation]. +// One and only one of Successful or Pending will be non-nil. +type ClientStartOperationResult[T any] struct { + // Set when start completes synchronously and successfully. + // + // If T is a [LazyValue], ensure that your consume it or read the underlying content in its entirety and close it to + // free up the underlying connection. + Successful T + // Set when the handler indicates that it started an asynchronous operation. + // The attached handle can be used to perform actions such as cancel the operation or get its result. + Pending *OperationHandle[T] + // Links contain information about the operations done by the handler. + Links []Link +} + +// StartOperation calls the configured Nexus endpoint to start an operation. +// +// This method has the following possible outcomes: +// +// 1. The operation completes successfully. The result of this call will be set as a [LazyValue] in +// ClientStartOperationResult.Successful and must be consumed to free up the underlying connection. +// +// 2. The operation was started and the handler has indicated that it will complete asynchronously. An +// [OperationHandle] will be returned as ClientStartOperationResult.Pending, which can be used to perform actions +// such as getting its result. +// +// 3. The operation was unsuccessful. The returned result will be nil and error will be an +// [UnsuccessfulOperationError]. +// +// 4. Any other error. +func (c *Client) StartOperation( + ctx context.Context, + operation string, + input any, + options StartOperationOptions, +) (*ClientStartOperationResult[*LazyValue], error) { + var reader *Reader + if r, ok := input.(*Reader); ok { + // Close the input reader in case we error before sending the HTTP request (which may double close but + // that's fine since we ignore the error). + defer r.Close() + reader = r + } else { + content, ok := input.(*Content) + if !ok { + var err error + content, err = c.options.Serializer.Serialize(input) + if err != nil { + return nil, err + } + } + header := maps.Clone(content.Header) + if header == nil { + header = make(Header, 1) + } + header["length"] = strconv.Itoa(len(content.Data)) + + reader = &Reader{ + io.NopCloser(bytes.NewReader(content.Data)), + header, + } + } + + url := c.serviceBaseURL.JoinPath(url.PathEscape(c.options.Service), url.PathEscape(operation)) + + if options.CallbackURL != "" { + q := url.Query() + q.Set(queryCallbackURL, options.CallbackURL) + url.RawQuery = q.Encode() + } + request, err := http.NewRequestWithContext(ctx, "POST", url.String(), reader) + if err != nil { + return nil, err + } + + if options.RequestID == "" { + options.RequestID = uuid.NewString() + } + request.Header.Set(headerRequestID, options.RequestID) + request.Header.Set(headerUserAgent, userAgent) + addContentHeaderToHTTPHeader(reader.Header, request.Header) + addCallbackHeaderToHTTPHeader(options.CallbackHeader, request.Header) + if err := addLinksToHTTPHeader(options.Links, request.Header); err != nil { + return nil, fmt.Errorf("failed to serialize links into header: %w", err) + } + addContextTimeoutToHTTPHeader(ctx, request.Header) + addNexusHeaderToHTTPHeader(options.Header, request.Header) + + response, err := c.options.HTTPCaller(request) + if err != nil { + return nil, err + } + // Do not close response body here to allow successful result to read it. + if response.StatusCode == http.StatusOK { + return &ClientStartOperationResult[*LazyValue]{ + Successful: &LazyValue{ + serializer: c.options.Serializer, + Reader: &Reader{ + response.Body, + prefixStrippedHTTPHeaderToNexusHeader(response.Header, "content-"), + }, + }, + }, nil + } + + // Do this once here and make sure it doesn't leak. + body, err := readAndReplaceBody(response) + if err != nil { + return nil, err + } + + switch response.StatusCode { + case http.StatusCreated: + info, err := operationInfoFromResponse(response, body) + if err != nil { + return nil, err + } + if info.State != OperationStateRunning { + return nil, newUnexpectedResponseError(fmt.Sprintf("invalid operation state in response info: %q", info.State), response, body) + } + links, err := getLinksFromHeader(response.Header) + if err != nil { + return nil, fmt.Errorf( + "%w: %w", + newUnexpectedResponseError( + fmt.Sprintf("invalid links header: %q", response.Header.Values(headerLink)), + response, + body, + ), + err, + ) + } + return &ClientStartOperationResult[*LazyValue]{ + Pending: &OperationHandle[*LazyValue]{ + Operation: operation, + ID: info.ID, + client: c, + }, + Links: links, + }, nil + case statusOperationFailed: + state, err := getUnsuccessfulStateFromHeader(response, body) + if err != nil { + return nil, err + } + + failure, err := failureFromResponse(response, body) + if err != nil { + return nil, err + } + + return nil, &UnsuccessfulOperationError{ + State: state, + Failure: failure, + } + default: + return nil, newUnexpectedResponseError(fmt.Sprintf("unexpected response status: %q", response.Status), response, body) + } +} + +// ExecuteOperationOptions are options for [Client.ExecuteOperation]. +type ExecuteOperationOptions struct { + // Callback URL to provide to the handle for receiving async operation completions. Optional. + // Even though Client.ExecuteOperation waits for operation completion, some applications may want to set this + // callback as a fallback mechanism. + CallbackURL string + // Optional header fields set by a client that are required to be attached to the callback request when an + // asynchronous operation completes. + CallbackHeader Header + // Request ID that may be used by the server handler to dedupe this start request. + // By default a v4 UUID will be generated by the client. + RequestID string + // Links contain arbitrary caller information. Handlers may use these links as + // metadata on resources associated with and operation. + Links []Link + // Header to attach to start and get-result requests. Optional. + // + // Header values set here will overwrite any SDK-provided values for the same key. + // + // Header keys with the "content-" prefix are reserved for [Serializer] headers and should not be set in the + // client API; they are not available to server [Handler] and [Operation] implementations. + Header Header + // Duration to wait for operation completion. + // + // ⚠ NOTE: unlike GetOperationResultOptions.Wait, zero and negative values are considered effectively infinite. + Wait time.Duration +} + +// ExecuteOperation is a helper for starting an operation and waiting for its completion. +// +// For asynchronous operations, the client will long poll for their result, issuing one or more requests until the +// wait period provided via [ExecuteOperationOptions] exceeds, in which case an [ErrOperationStillRunning] error is +// returned. +// +// The wait time is capped to the deadline of the provided context. Make sure to handle both context deadline errors and +// [ErrOperationStillRunning]. +// +// Note that the wait period is enforced by the server and may not be respected if the server is misbehaving. Set the +// context deadline to the max allowed wait period to ensure this call returns in a timely fashion. +// +// ⚠️ If this method completes successfully, the returned response's body must be read in its entirety and closed to +// free up the underlying connection. +func (c *Client) ExecuteOperation(ctx context.Context, operation string, input any, options ExecuteOperationOptions) (*LazyValue, error) { + so := StartOperationOptions{ + CallbackURL: options.CallbackURL, + CallbackHeader: options.CallbackHeader, + RequestID: options.RequestID, + Links: options.Links, + Header: options.Header, + } + result, err := c.StartOperation(ctx, operation, input, so) + if err != nil { + return nil, err + } + if result.Successful != nil { + return result.Successful, nil + } + handle := result.Pending + gro := GetOperationResultOptions{ + Header: options.Header, + } + if options.Wait <= 0 { + gro.Wait = time.Duration(math.MaxInt64) + } else { + gro.Wait = options.Wait + } + return handle.GetResult(ctx, gro) +} + +// NewHandle gets a handle to an asynchronous operation by name and ID. +// Does not incur a trip to the server. +// Fails if provided an empty operation or ID. +func (c *Client) NewHandle(operation string, operationID string) (*OperationHandle[*LazyValue], error) { + var es []error + if operation == "" { + es = append(es, errEmptyOperationName) + } + if operationID == "" { + es = append(es, errEmptyOperationID) + } + if len(es) > 0 { + return nil, errors.Join(es...) + } + return &OperationHandle[*LazyValue]{ + client: c, + Operation: operation, + ID: operationID, + }, nil +} + +// readAndReplaceBody reads the response body in its entirety and closes it, and then replaces the original response +// body with an in-memory buffer. +// The body is replaced even when there was an error reading the entire body. +func readAndReplaceBody(response *http.Response) ([]byte, error) { + responseBody := response.Body + body, err := io.ReadAll(responseBody) + responseBody.Close() + response.Body = io.NopCloser(bytes.NewReader(body)) + return body, err +} + +func operationInfoFromResponse(response *http.Response, body []byte) (*OperationInfo, error) { + if !isMediaTypeJSON(response.Header.Get("Content-Type")) { + return nil, newUnexpectedResponseError(fmt.Sprintf("invalid response content type: %q", response.Header.Get("Content-Type")), response, body) + } + var info OperationInfo + if err := json.Unmarshal(body, &info); err != nil { + return nil, err + } + return &info, nil +} + +func failureFromResponse(response *http.Response, body []byte) (Failure, error) { + if !isMediaTypeJSON(response.Header.Get("Content-Type")) { + return Failure{}, newUnexpectedResponseError(fmt.Sprintf("invalid response content type: %q", response.Header.Get("Content-Type")), response, body) + } + var failure Failure + err := json.Unmarshal(body, &failure) + return failure, err +} + +func getUnsuccessfulStateFromHeader(response *http.Response, body []byte) (OperationState, error) { + state := OperationState(response.Header.Get(headerOperationState)) + switch state { + case OperationStateCanceled: + return state, nil + case OperationStateFailed: + return state, nil + default: + return state, newUnexpectedResponseError(fmt.Sprintf("invalid operation state header: %q", state), response, body) + } +} diff --git a/vendor/github.com/nexus-rpc/sdk-go/nexus/completion.go b/vendor/github.com/nexus-rpc/sdk-go/nexus/completion.go new file mode 100644 index 00000000000..0dc50802efe --- /dev/null +++ b/vendor/github.com/nexus-rpc/sdk-go/nexus/completion.go @@ -0,0 +1,207 @@ +package nexus + +import ( + "bytes" + "context" + "encoding/json" + "io" + "log/slog" + "net/http" + "strconv" +) + +// NewCompletionHTTPRequest creates an HTTP request deliver an operation completion to a given URL. +func NewCompletionHTTPRequest(ctx context.Context, url string, completion OperationCompletion) (*http.Request, error) { + httpReq, err := http.NewRequestWithContext(ctx, "POST", url, nil) + if err != nil { + return nil, err + } + if err := completion.applyToHTTPRequest(httpReq); err != nil { + return nil, err + } + + httpReq.Header.Set(headerUserAgent, userAgent) + return httpReq, nil +} + +// OperationCompletion is input for [NewCompletionHTTPRequest]. +// It has two implementations: [OperationCompletionSuccessful] and [OperationCompletionUnsuccessful]. +type OperationCompletion interface { + applyToHTTPRequest(*http.Request) error +} + +// OperationCompletionSuccessful is input for [NewCompletionHTTPRequest], used to deliver successful operation results. +type OperationCompletionSuccessful struct { + // Header to send in the completion request. + Header http.Header + // Body to send in the completion HTTP request. + // If it implements `io.Closer` it will automatically be closed by the client. + Body io.Reader +} + +// OperationCompletionSuccesfulOptions are options for [NewOperationCompletionSuccessful]. +type OperationCompletionSuccesfulOptions struct { + // Optional serializer for the result. Defaults to the SDK's default Serializer, which handles JSONables, byte + // slices and nils. + Serializer Serializer +} + +// NewOperationCompletionSuccessful constructs an [OperationCompletionSuccessful] from a given result. +func NewOperationCompletionSuccessful(result any, options OperationCompletionSuccesfulOptions) (*OperationCompletionSuccessful, error) { + if reader, ok := result.(*Reader); ok { + return &OperationCompletionSuccessful{ + Header: addContentHeaderToHTTPHeader(reader.Header, make(http.Header)), + Body: reader.ReadCloser, + }, nil + } else { + content, ok := result.(*Content) + if !ok { + var err error + serializer := options.Serializer + if serializer == nil { + serializer = defaultSerializer + } + content, err = serializer.Serialize(result) + if err != nil { + return nil, err + } + } + header := http.Header{"Content-Length": []string{strconv.Itoa(len(content.Data))}} + + return &OperationCompletionSuccessful{ + Header: addContentHeaderToHTTPHeader(content.Header, header), + Body: bytes.NewReader(content.Data), + }, nil + } +} + +func (c *OperationCompletionSuccessful) applyToHTTPRequest(request *http.Request) error { + if c.Header != nil { + request.Header = c.Header.Clone() + } + request.Header.Set(headerOperationState, string(OperationStateSucceeded)) + if closer, ok := c.Body.(io.ReadCloser); ok { + request.Body = closer + } else { + request.Body = io.NopCloser(c.Body) + } + return nil +} + +// OperationCompletionUnsuccessful is input for [NewCompletionHTTPRequest], used to deliver unsuccessful operation +// results. +type OperationCompletionUnsuccessful struct { + // Header to send in the completion request. + Header http.Header + // State of the operation, should be failed or canceled. + State OperationState + // Failure object to send with the completion. + Failure *Failure +} + +func (c *OperationCompletionUnsuccessful) applyToHTTPRequest(request *http.Request) error { + if c.Header != nil { + request.Header = c.Header.Clone() + } + request.Header.Set(headerOperationState, string(c.State)) + request.Header.Set("Content-Type", contentTypeJSON) + + b, err := json.Marshal(c.Failure) + if err != nil { + return err + } + + request.Body = io.NopCloser(bytes.NewReader(b)) + return nil +} + +// CompletionRequest is input for CompletionHandler.CompleteOperation. +type CompletionRequest struct { + // The original HTTP request. + HTTPRequest *http.Request + // State of the operation. + State OperationState + // Parsed from request and set if State is failed or canceled. + Failure *Failure + // Extracted from request and set if State is succeeded. + Result *LazyValue +} + +// A CompletionHandler can receive operation completion requests as delivered via the callback URL provided in +// start-operation requests. +type CompletionHandler interface { + CompleteOperation(context.Context, *CompletionRequest) error +} + +// CompletionHandlerOptions are options for [NewCompletionHTTPHandler]. +type CompletionHandlerOptions struct { + // Handler for completion requests. + Handler CompletionHandler + // A stuctured logging handler. + // Defaults to slog.Default(). + Logger *slog.Logger + // A [Serializer] to customize handler serialization behavior. + // By default the handler handles, JSONables, byte slices, and nil. + Serializer Serializer +} + +type completionHTTPHandler struct { + baseHTTPHandler + options CompletionHandlerOptions +} + +func (h *completionHTTPHandler) ServeHTTP(writer http.ResponseWriter, request *http.Request) { + ctx := request.Context() + completion := CompletionRequest{ + State: OperationState(request.Header.Get(headerOperationState)), + HTTPRequest: request, + } + switch completion.State { + case OperationStateFailed, OperationStateCanceled: + if !isMediaTypeJSON(request.Header.Get("Content-Type")) { + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "invalid request content type: %q", request.Header.Get("Content-Type"))) + return + } + var failure Failure + b, err := io.ReadAll(request.Body) + if err != nil { + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "failed to read Failure from request body")) + return + } + if err := json.Unmarshal(b, &failure); err != nil { + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "failed to read Failure from request body")) + return + } + completion.Failure = &failure + case OperationStateSucceeded: + completion.Result = &LazyValue{ + serializer: h.options.Serializer, + Reader: &Reader{ + request.Body, + prefixStrippedHTTPHeaderToNexusHeader(request.Header, "content-"), + }, + } + default: + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "invalid request operation state: %q", completion.State)) + return + } + if err := h.options.Handler.CompleteOperation(ctx, &completion); err != nil { + h.writeFailure(writer, err) + } +} + +// NewCompletionHTTPHandler constructs an [http.Handler] from given options for handling operation completion requests. +func NewCompletionHTTPHandler(options CompletionHandlerOptions) http.Handler { + if options.Logger == nil { + options.Logger = slog.Default() + } + if options.Serializer == nil { + options.Serializer = defaultSerializer + } + return &completionHTTPHandler{ + options: options, + baseHTTPHandler: baseHTTPHandler{ + logger: options.Logger, + }, + } +} diff --git a/vendor/github.com/nexus-rpc/sdk-go/nexus/handle.go b/vendor/github.com/nexus-rpc/sdk-go/nexus/handle.go new file mode 100644 index 00000000000..c0427332003 --- /dev/null +++ b/vendor/github.com/nexus-rpc/sdk-go/nexus/handle.go @@ -0,0 +1,189 @@ +package nexus + +import ( + "context" + "errors" + "fmt" + "net/http" + "net/url" + "time" +) + +const getResultContextPadding = time.Second * 5 + +// An OperationHandle is used to cancel operations and get their result and status. +type OperationHandle[T any] struct { + // Name of the Operation this handle represents. + Operation string + // Handler generated ID for this handle's operation. + ID string + client *Client +} + +// GetInfo gets operation information, issuing a network request to the service handler. +func (h *OperationHandle[T]) GetInfo(ctx context.Context, options GetOperationInfoOptions) (*OperationInfo, error) { + url := h.client.serviceBaseURL.JoinPath(url.PathEscape(h.client.options.Service), url.PathEscape(h.Operation), url.PathEscape(h.ID)) + request, err := http.NewRequestWithContext(ctx, "GET", url.String(), nil) + if err != nil { + return nil, err + } + addContextTimeoutToHTTPHeader(ctx, request.Header) + addNexusHeaderToHTTPHeader(options.Header, request.Header) + + request.Header.Set(headerUserAgent, userAgent) + response, err := h.client.options.HTTPCaller(request) + if err != nil { + return nil, err + } + + // Do this once here and make sure it doesn't leak. + body, err := readAndReplaceBody(response) + if err != nil { + return nil, err + } + + if response.StatusCode != http.StatusOK { + return nil, newUnexpectedResponseError(fmt.Sprintf("unexpected response status: %q", response.Status), response, body) + } + + return operationInfoFromResponse(response, body) +} + +// GetResult gets the result of an operation, issuing a network request to the service handler. +// +// By default, GetResult returns (nil, [ErrOperationStillRunning]) immediately after issuing a call if the operation has +// not yet completed. +// +// Callers may set GetOperationResultOptions.Wait to a value greater than 0 to alter this behavior, causing the client +// to long poll for the result issuing one or more requests until the provided wait period exceeds, in which case (nil, +// [ErrOperationStillRunning]) is returned. +// +// The wait time is capped to the deadline of the provided context. Make sure to handle both context deadline errors and +// [ErrOperationStillRunning]. +// +// Note that the wait period is enforced by the server and may not be respected if the server is misbehaving. Set the +// context deadline to the max allowed wait period to ensure this call returns in a timely fashion. +// +// ⚠️ If a [LazyValue] is returned (as indicated by T), it must be consumed to free up the underlying connection. +func (h *OperationHandle[T]) GetResult(ctx context.Context, options GetOperationResultOptions) (T, error) { + var result T + url := h.client.serviceBaseURL.JoinPath(url.PathEscape(h.client.options.Service), url.PathEscape(h.Operation), url.PathEscape(h.ID), "result") + request, err := http.NewRequestWithContext(ctx, "GET", url.String(), nil) + if err != nil { + return result, err + } + addContextTimeoutToHTTPHeader(ctx, request.Header) + request.Header.Set(headerUserAgent, userAgent) + addNexusHeaderToHTTPHeader(options.Header, request.Header) + + startTime := time.Now() + wait := options.Wait + for { + if wait > 0 { + if deadline, set := ctx.Deadline(); set { + // Ensure we don't wait longer than the deadline but give some buffer prevent racing between wait and + // context deadline. + wait = min(wait, time.Until(deadline)+getResultContextPadding) + } + + q := request.URL.Query() + q.Set(queryWait, fmt.Sprintf("%dms", wait.Milliseconds())) + request.URL.RawQuery = q.Encode() + } else { + // We may reuse the request object multiple times and will need to reset the query when wait becomes 0 or + // negative. + request.URL.RawQuery = "" + } + + response, err := h.sendGetOperationRequest(ctx, request) + if err != nil { + if wait > 0 && errors.Is(err, errOperationWaitTimeout) { + // TODO: Backoff a bit in case the server is continually returning timeouts due to some LB configuration + // issue to avoid blowing it up with repeated calls. + wait = options.Wait - time.Since(startTime) + continue + } + return result, err + } + s := &LazyValue{ + serializer: h.client.options.Serializer, + Reader: &Reader{ + response.Body, + prefixStrippedHTTPHeaderToNexusHeader(response.Header, "content-"), + }, + } + if _, ok := any(result).(*LazyValue); ok { + return any(s).(T), nil + } else { + return result, s.Consume(&result) + } + } +} + +func (h *OperationHandle[T]) sendGetOperationRequest(ctx context.Context, request *http.Request) (*http.Response, error) { + response, err := h.client.options.HTTPCaller(request) + if err != nil { + return nil, err + } + + if response.StatusCode == http.StatusOK { + return response, nil + } + + // Do this once here and make sure it doesn't leak. + body, err := readAndReplaceBody(response) + if err != nil { + return nil, err + } + + switch response.StatusCode { + case http.StatusRequestTimeout: + return nil, errOperationWaitTimeout + case statusOperationRunning: + return nil, ErrOperationStillRunning + case statusOperationFailed: + state, err := getUnsuccessfulStateFromHeader(response, body) + if err != nil { + return nil, err + } + failure, err := failureFromResponse(response, body) + if err != nil { + return nil, err + } + return nil, &UnsuccessfulOperationError{ + State: state, + Failure: failure, + } + default: + return nil, newUnexpectedResponseError(fmt.Sprintf("unexpected response status: %q", response.Status), response, body) + } +} + +// Cancel requests to cancel an asynchronous operation. +// +// Cancelation is asynchronous and may be not be respected by the operation's implementation. +func (h *OperationHandle[T]) Cancel(ctx context.Context, options CancelOperationOptions) error { + url := h.client.serviceBaseURL.JoinPath(url.PathEscape(h.client.options.Service), url.PathEscape(h.Operation), url.PathEscape(h.ID), "cancel") + request, err := http.NewRequestWithContext(ctx, "POST", url.String(), nil) + if err != nil { + return err + } + addContextTimeoutToHTTPHeader(ctx, request.Header) + request.Header.Set(headerUserAgent, userAgent) + addNexusHeaderToHTTPHeader(options.Header, request.Header) + response, err := h.client.options.HTTPCaller(request) + if err != nil { + return err + } + + // Do this once here and make sure it doesn't leak. + body, err := readAndReplaceBody(response) + if err != nil { + return err + } + + if response.StatusCode != http.StatusAccepted { + return newUnexpectedResponseError(fmt.Sprintf("unexpected response status: %q", response.Status), response, body) + } + return nil +} diff --git a/vendor/github.com/nexus-rpc/sdk-go/nexus/operation.go b/vendor/github.com/nexus-rpc/sdk-go/nexus/operation.go new file mode 100644 index 00000000000..9e574959c29 --- /dev/null +++ b/vendor/github.com/nexus-rpc/sdk-go/nexus/operation.go @@ -0,0 +1,344 @@ +package nexus + +import ( + "context" + "errors" + "fmt" + "reflect" + "strings" +) + +// NoValue is a marker type for an operations that do not accept any input or return a value (nil). +// +// nexus.NewSyncOperation("my-empty-operation", func(context.Context, nexus.NoValue, options, nexus.StartOperationOptions) (nexus.NoValue, error) { +// return nil, nil +// )} +type NoValue *struct{} + +// OperationReference provides a typed interface for invoking operations. Every [Operation] is also an +// [OperationReference]. Callers may create references using [NewOperationReference] when the implementation is not +// available. +type OperationReference[I, O any] interface { + Name() string + // A type inference helper for implementations of this interface. + inferType(I, O) +} + +type operationReference[I, O any] string + +// NewOperationReference creates an [OperationReference] with the provided type parameters and name. +// It provides typed interface for invoking operations when the implementation is not available to the caller. +func NewOperationReference[I, O any](name string) OperationReference[I, O] { + return operationReference[I, O](name) +} + +func (r operationReference[I, O]) Name() string { + return string(r) +} + +func (operationReference[I, O]) inferType(I, O) {} //nolint:unused + +// A RegisterableOperation is accepted in [OperationRegistry.Register]. +// Embed [UnimplementedOperation] to implement it. +type RegisterableOperation interface { + // Name of the operation. Used for invocation and registration. + Name() string + mustEmbedUnimplementedOperation() +} + +// Operation is a handler for a single operation. +// +// Operation implementations must embed the [UnimplementedOperation]. +// +// All Operation methods can return a [HandlerError] to fail requests with a custom [HandlerErrorType] and structured [Failure]. +// Arbitrary errors from handler methods are turned into [HandlerErrorTypeInternal],their details are logged and hidden +// from the caller. +type Operation[I, O any] interface { + RegisterableOperation + OperationReference[I, O] + + // Start handles requests for starting an operation. Return [HandlerStartOperationResultSync] to respond + // successfully - inline, or [HandlerStartOperationResultAsync] to indicate that an asynchronous operation was + // started. Return an [UnsuccessfulOperationError] to indicate that an operation completed as failed or + // canceled. + Start(context.Context, I, StartOperationOptions) (HandlerStartOperationResult[O], error) + // GetResult handles requests to get the result of an asynchronous operation. Return non error result to respond + // successfully - inline, or error with [ErrOperationStillRunning] to indicate that an asynchronous operation is + // still running. Return an [UnsuccessfulOperationError] to indicate that an operation completed as failed or + // canceled. + // + // When [GetOperationResultOptions.Wait] is greater than zero, this request should be treated as a long poll. + // Long poll requests have a server side timeout, configurable via [HandlerOptions.GetResultTimeout], and exposed + // via context deadline. The context deadline is decoupled from the application level Wait duration. + // + // It is the implementor's responsiblity to respect the client's wait duration and return in a timely fashion. + // Consider using a derived context that enforces the wait timeout when implementing this method and return + // [ErrOperationStillRunning] when that context expires as shown in the [Handler] example. + GetResult(context.Context, string, GetOperationResultOptions) (O, error) + // GetInfo handles requests to get information about an asynchronous operation. + GetInfo(context.Context, string, GetOperationInfoOptions) (*OperationInfo, error) + // Cancel handles requests to cancel an asynchronous operation. + // Cancelation in Nexus is: + // 1. asynchronous - returning from this method only ensures that cancelation is delivered, it may later be + // ignored by the underlying operation implemention. + // 2. idempotent - implementors should ignore duplicate cancelations for the same operation. + Cancel(context.Context, string, CancelOperationOptions) error +} + +type syncOperation[I, O any] struct { + UnimplementedOperation[I, O] + + Handler func(context.Context, I, StartOperationOptions) (O, error) + name string +} + +// NewSyncOperation is a helper for creating a synchronous-only [Operation] from a given name and handler function. +func NewSyncOperation[I, O any](name string, handler func(context.Context, I, StartOperationOptions) (O, error)) Operation[I, O] { + return &syncOperation[I, O]{ + name: name, + Handler: handler, + } +} + +// Name implements Operation. +func (h *syncOperation[I, O]) Name() string { + return h.name +} + +// Start implements Operation. +func (h *syncOperation[I, O]) Start(ctx context.Context, input I, options StartOperationOptions) (HandlerStartOperationResult[O], error) { + o, err := h.Handler(ctx, input, options) + if err != nil { + return nil, err + } + return &HandlerStartOperationResultSync[O]{o}, err +} + +// A Service is a container for a group of operations. +type Service struct { + Name string + + operations map[string]RegisterableOperation +} + +// NewService constructs a [Service]. +func NewService(name string) *Service { + return &Service{ + Name: name, + operations: make(map[string]RegisterableOperation), + } +} + +// Register one or more operations. +// Returns an error if duplicate operations were registered with the same name or when trying to register an operation +// with no name. +// +// Can be called multiple times and is not thread safe. +func (s *Service) Register(operations ...RegisterableOperation) error { + var dups []string + for _, op := range operations { + if op.Name() == "" { + return fmt.Errorf("tried to register an operation with no name") + } + if _, found := s.operations[op.Name()]; found { + dups = append(dups, op.Name()) + } else { + s.operations[op.Name()] = op + } + } + if len(dups) > 0 { + return fmt.Errorf("duplicate operations: %s", strings.Join(dups, ", ")) + } + return nil +} + +// A ServiceRegistry registers services and constructs a [Handler] that dispatches operations requests to those services. +type ServiceRegistry struct { + services map[string]*Service +} + +func NewServiceRegistry() *ServiceRegistry { + return &ServiceRegistry{services: make(map[string]*Service)} +} + +// Register one or more service. +// Returns an error if duplicate operations were registered with the same name or when trying to register a service with +// no name. +// +// Can be called multiple times and is not thread safe. +func (r *ServiceRegistry) Register(services ...*Service) error { + var dups []string + for _, service := range services { + if service.Name == "" { + return fmt.Errorf("tried to register a service with no name") + } + if _, found := r.services[service.Name]; found { + dups = append(dups, service.Name) + } else { + r.services[service.Name] = service + } + } + if len(dups) > 0 { + return fmt.Errorf("duplicate services: %s", strings.Join(dups, ", ")) + } + return nil +} + +// NewHandler creates a [Handler] that dispatches requests to registered operations based on their name. +func (r *ServiceRegistry) NewHandler() (Handler, error) { + if len(r.services) == 0 { + return nil, errors.New("must register at least one service") + } + for _, service := range r.services { + if len(service.operations) == 0 { + return nil, fmt.Errorf("service %q has no operations registered", service.Name) + } + } + + return ®istryHandler{services: r.services}, nil +} + +type registryHandler struct { + UnimplementedHandler + + services map[string]*Service +} + +// CancelOperation implements Handler. +func (r *registryHandler) CancelOperation(ctx context.Context, service, operation string, operationID string, options CancelOperationOptions) error { + s, ok := r.services[service] + if !ok { + return HandlerErrorf(HandlerErrorTypeNotFound, "service %q not found", service) + } + h, ok := s.operations[operation] + if !ok { + return HandlerErrorf(HandlerErrorTypeNotFound, "operation %q not found", operation) + } + + // NOTE: We could avoid reflection here if we put the Cancel method on RegisterableOperation but it doesn't seem + // worth it since we need reflection for the generic methods. + m, _ := reflect.TypeOf(h).MethodByName("Cancel") + values := m.Func.Call([]reflect.Value{reflect.ValueOf(h), reflect.ValueOf(ctx), reflect.ValueOf(operationID), reflect.ValueOf(options)}) + if values[0].IsNil() { + return nil + } + return values[0].Interface().(error) +} + +// GetOperationInfo implements Handler. +func (r *registryHandler) GetOperationInfo(ctx context.Context, service, operation string, operationID string, options GetOperationInfoOptions) (*OperationInfo, error) { + s, ok := r.services[service] + if !ok { + return nil, HandlerErrorf(HandlerErrorTypeNotFound, "service %q not found", service) + } + h, ok := s.operations[operation] + if !ok { + return nil, HandlerErrorf(HandlerErrorTypeNotFound, "operation %q not found", operation) + } + + // NOTE: We could avoid reflection here if we put the Cancel method on RegisterableOperation but it doesn't seem + // worth it since we need reflection for the generic methods. + m, _ := reflect.TypeOf(h).MethodByName("GetInfo") + values := m.Func.Call([]reflect.Value{reflect.ValueOf(h), reflect.ValueOf(ctx), reflect.ValueOf(operationID), reflect.ValueOf(options)}) + if !values[1].IsNil() { + return nil, values[1].Interface().(error) + } + ret := values[0].Interface() + return ret.(*OperationInfo), nil +} + +// GetOperationResult implements Handler. +func (r *registryHandler) GetOperationResult(ctx context.Context, service, operation string, operationID string, options GetOperationResultOptions) (any, error) { + s, ok := r.services[service] + if !ok { + return nil, HandlerErrorf(HandlerErrorTypeNotFound, "service %q not found", service) + } + h, ok := s.operations[operation] + if !ok { + return nil, HandlerErrorf(HandlerErrorTypeNotFound, "operation %q not found", operation) + } + + m, _ := reflect.TypeOf(h).MethodByName("GetResult") + values := m.Func.Call([]reflect.Value{reflect.ValueOf(h), reflect.ValueOf(ctx), reflect.ValueOf(operationID), reflect.ValueOf(options)}) + if !values[1].IsNil() { + return nil, values[1].Interface().(error) + } + ret := values[0].Interface() + return ret, nil +} + +// StartOperation implements Handler. +func (r *registryHandler) StartOperation(ctx context.Context, service, operation string, input *LazyValue, options StartOperationOptions) (HandlerStartOperationResult[any], error) { + s, ok := r.services[service] + if !ok { + return nil, HandlerErrorf(HandlerErrorTypeNotFound, "service %q not found", service) + } + h, ok := s.operations[operation] + if !ok { + return nil, HandlerErrorf(HandlerErrorTypeNotFound, "operation %q not found", operation) + } + + m, _ := reflect.TypeOf(h).MethodByName("Start") + inputType := m.Type.In(2) + iptr := reflect.New(inputType).Interface() + if err := input.Consume(iptr); err != nil { + // TODO: log the error? Do we need to accept a logger for this single line? + return nil, HandlerErrorf(HandlerErrorTypeBadRequest, "invalid input") + } + i := reflect.ValueOf(iptr).Elem() + + values := m.Func.Call([]reflect.Value{reflect.ValueOf(h), reflect.ValueOf(ctx), i, reflect.ValueOf(options)}) + if !values[1].IsNil() { + return nil, values[1].Interface().(error) + } + ret := values[0].Interface() + return ret.(HandlerStartOperationResult[any]), nil +} + +var _ Handler = ®istryHandler{} + +// ExecuteOperation is the type safe version of [Client.ExecuteOperation]. +// It accepts input of type I and returns output of type O, removing the need to consume the [LazyValue] returned by the +// client method. +// +// ref := NewOperationReference[MyInput, MyOutput]("my-operation") +// out, err := ExecuteOperation(ctx, client, ref, MyInput{}, options) // returns MyOutput, error +func ExecuteOperation[I, O any](ctx context.Context, client *Client, operation OperationReference[I, O], input I, request ExecuteOperationOptions) (O, error) { + var o O + value, err := client.ExecuteOperation(ctx, operation.Name(), input, request) + if err != nil { + return o, err + } + return o, value.Consume(&o) +} + +// StartOperation is the type safe version of [Client.StartOperation]. +// It accepts input of type I and returns a [ClientStartOperationResult] of type O, removing the need to consume the +// [LazyValue] returned by the client method. +func StartOperation[I, O any](ctx context.Context, client *Client, operation OperationReference[I, O], input I, request StartOperationOptions) (*ClientStartOperationResult[O], error) { + result, err := client.StartOperation(ctx, operation.Name(), input, request) + if err != nil { + return nil, err + } + if result.Successful != nil { + var o O + if err := result.Successful.Consume(&o); err != nil { + return nil, err + } + return &ClientStartOperationResult[O]{Successful: o}, nil + } + handle := OperationHandle[O]{client: client, Operation: operation.Name(), ID: result.Pending.ID} + return &ClientStartOperationResult[O]{ + Pending: &handle, + Links: result.Links, + }, nil +} + +// NewHandle is the type safe version of [Client.NewHandle]. +// The [Handle.GetResult] method will return an output of type O. +func NewHandle[I, O any](client *Client, operation OperationReference[I, O], operationID string) (*OperationHandle[O], error) { + if operationID == "" { + return nil, errEmptyOperationID + } + return &OperationHandle[O]{client: client, Operation: operation.Name(), ID: operationID}, nil +} diff --git a/vendor/github.com/nexus-rpc/sdk-go/nexus/options.go b/vendor/github.com/nexus-rpc/sdk-go/nexus/options.go new file mode 100644 index 00000000000..9e1c6182ab9 --- /dev/null +++ b/vendor/github.com/nexus-rpc/sdk-go/nexus/options.go @@ -0,0 +1,65 @@ +package nexus + +import ( + "time" +) + +// StartOperationOptions are options for the StartOperation client and server APIs. +type StartOperationOptions struct { + // Header contains the request header fields either received by the server or to be sent by the client. + // + // Header will always be non empty in server methods and can be optionally set in the client API. + // + // Header values set here will overwrite any SDK-provided values for the same key. + // + // Header keys with the "content-" prefix are reserved for [Serializer] headers and should not be set in the + // client API; they are not available to server [Handler] and [Operation] implementations. + Header Header + // Callbacks are used to deliver completion of async operations. + // This value may optionally be set by the client and should be called by a handler upon completion if the started operation is async. + // + // Implement a [CompletionHandler] and expose it as an HTTP handler to handle async completions. + CallbackURL string + // Optional header fields set by a client that are required to be attached to the callback request when an + // asynchronous operation completes. + CallbackHeader Header + // Request ID that may be used by the server handler to dedupe a start request. + // By default a v4 UUID will be generated by the client. + RequestID string + // Links contain arbitrary caller information. Handlers may use these links as + // metadata on resources associated with and operation. + Links []Link +} + +// GetOperationResultOptions are options for the GetOperationResult client and server APIs. +type GetOperationResultOptions struct { + // Header contains the request header fields either received by the server or to be sent by the client. + // + // Header will always be non empty in server methods and can be optionally set in the client API. + // + // Header values set here will overwrite any SDK-provided values for the same key. + Header Header + // If non-zero, reflects the duration the caller has indicated that it wants to wait for operation completion, + // turning the request into a long poll. + Wait time.Duration +} + +// GetOperationInfoOptions are options for the GetOperationInfo client and server APIs. +type GetOperationInfoOptions struct { + // Header contains the request header fields either received by the server or to be sent by the client. + // + // Header will always be non empty in server methods and can be optionally set in the client API. + // + // Header values set here will overwrite any SDK-provided values for the same key. + Header Header +} + +// CancelOperationOptions are options for the CancelOperation client and server APIs. +type CancelOperationOptions struct { + // Header contains the request header fields either received by the server or to be sent by the client. + // + // Header will always be non empty in server methods and can be optionally set in the client API. + // + // Header values set here will overwrite any SDK-provided values for the same key. + Header Header +} diff --git a/vendor/github.com/nexus-rpc/sdk-go/nexus/serializer.go b/vendor/github.com/nexus-rpc/sdk-go/nexus/serializer.go new file mode 100644 index 00000000000..cd5edf5e7b0 --- /dev/null +++ b/vendor/github.com/nexus-rpc/sdk-go/nexus/serializer.go @@ -0,0 +1,228 @@ +package nexus + +import ( + "encoding/json" + "errors" + "fmt" + "io" + "reflect" +) + +// A Reader is a container for a [Header] and an [io.Reader]. +// It is used to stream inputs and outputs in the various client and server APIs. +type Reader struct { + // ReaderCloser contains request or response data. May be nil for empty data. + io.ReadCloser + // Header that should include information on how to deserialize this content. + // Headers constructed by the framework always have lower case keys. + // User provided keys are considered case-insensitive by the framework. + Header Header +} + +// A Content is a container for a [Header] and a byte slice. +// It is used by the SDK's [Serializer] interface implementations. +type Content struct { + // Header that should include information on how to deserialize this content. + // Headers constructed by the framework always have lower case keys. + // User provided keys are considered case-insensitive by the framework. + Header Header + // Data contains request or response data. May be nil for empty data. + Data []byte +} + +// A LazyValue holds a value encoded in an underlying [Reader]. +// +// ⚠️ When a LazyValue is returned from a client - if directly accessing the [Reader] - it must be read it in its +// entirety and closed to free up the associated HTTP connection. Otherwise the [LazyValue.Consume] method must be +// called. +// +// ⚠️ When a LazyValue is passed to a server handler, it must not be used after the returning from the handler method. +type LazyValue struct { + serializer Serializer + Reader *Reader +} + +// Create a new [LazyValue] from a given serializer and reader. +func NewLazyValue(serializer Serializer, reader *Reader) *LazyValue { + return &LazyValue{ + serializer: serializer, + Reader: reader, + } +} + +// Consume consumes the lazy value, decodes it from the underlying [Reader], and stores the result in the value pointed +// to by v. +// +// var v int +// err := lazyValue.Consume(&v) +func (l *LazyValue) Consume(v any) error { + defer l.Reader.Close() + data, err := io.ReadAll(l.Reader) + if err != nil { + return err + } + return l.serializer.Deserialize(&Content{ + Header: l.Reader.Header, + Data: data, + }, v) +} + +// Serializer is used by the framework to serialize/deserialize input and output. +// To customize serialization logic, implement this interface and provide your implementation to framework methods such +// as [NewClient] and [NewHTTPHandler]. +// By default, the SDK supports serialization of JSONables, byte slices, and nils. +type Serializer interface { + // Serialize encodes a value into a [Content]. + Serialize(any) (*Content, error) + // Deserialize decodes a [Content] into a given reference. + Deserialize(*Content, any) error +} + +var anyType = reflect.TypeOf((*any)(nil)).Elem() + +var errSerializerIncompatible = errors.New("incompatible serializer") + +type serializerChain []Serializer + +func (c serializerChain) Serialize(v any) (*Content, error) { + for _, l := range c { + p, err := l.Serialize(v) + if err != nil { + if errors.Is(err, errSerializerIncompatible) { + continue + } + return nil, err + } + return p, nil + } + return nil, errSerializerIncompatible +} + +func (c serializerChain) Deserialize(content *Content, v any) error { + lenc := len(c) + for i := range c { + l := c[lenc-i-1] + if err := l.Deserialize(content, v); err != nil { + if errors.Is(err, errSerializerIncompatible) { + continue + } + return err + } + return nil + } + return errSerializerIncompatible +} + +var _ Serializer = serializerChain{} + +type jsonSerializer struct{} + +func (jsonSerializer) Deserialize(c *Content, v any) error { + if !isMediaTypeJSON(c.Header["type"]) { + return errSerializerIncompatible + } + return json.Unmarshal(c.Data, &v) +} + +func (jsonSerializer) Serialize(v any) (*Content, error) { + data, err := json.Marshal(v) + if err != nil { + return nil, err + } + return &Content{ + Header: Header{ + "type": "application/json", + }, + Data: data, + }, nil +} + +var _ Serializer = jsonSerializer{} + +type nilSerializer struct{} + +func (nilSerializer) Deserialize(c *Content, v any) error { + if len(c.Data) > 0 { + return errSerializerIncompatible + } + rv := reflect.ValueOf(v) + if rv.Kind() != reflect.Pointer { + return fmt.Errorf("cannot deserialize into non pointer: %v", v) + } + if rv.IsNil() { + return fmt.Errorf("cannot deserialize into nil pointer: %v", v) + } + re := rv.Elem() + if !re.CanSet() { + return fmt.Errorf("non settable type: %v", v) + } + // Set the zero value for the given type. + re.Set(reflect.Zero(re.Type())) + + return nil +} + +func (nilSerializer) Serialize(v any) (*Content, error) { + if v != nil { + rv := reflect.ValueOf(v) + if !(rv.Kind() == reflect.Pointer && rv.IsNil()) { + return nil, errSerializerIncompatible + } + } + return &Content{ + Header: Header{}, + Data: nil, + }, nil +} + +var _ Serializer = nilSerializer{} + +type byteSliceSerializer struct{} + +func (byteSliceSerializer) Deserialize(c *Content, v any) error { + if !isMediaTypeOctetStream(c.Header["type"]) { + return errSerializerIncompatible + } + if bPtr, ok := v.(*[]byte); ok { + if bPtr == nil { + return fmt.Errorf("cannot deserialize into nil pointer: %v", v) + } + *bPtr = c.Data + return nil + } + // v is *any + rv := reflect.ValueOf(v) + if rv.Kind() != reflect.Pointer { + return fmt.Errorf("cannot deserialize into non pointer: %v", v) + } + if rv.IsNil() { + return fmt.Errorf("cannot deserialize into nil pointer: %v", v) + } + if rv.Elem().Type() != anyType { + return fmt.Errorf("unsupported value type for content: %v", v) + } + rv.Elem().Set(reflect.ValueOf(c.Data)) + return nil +} + +func (byteSliceSerializer) Serialize(v any) (*Content, error) { + if b, ok := v.([]byte); ok { + return &Content{ + Header: Header{ + "type": "application/octet-stream", + }, + Data: b, + }, nil + } + return nil, errSerializerIncompatible +} + +var _ Serializer = byteSliceSerializer{} + +type compositeSerializer struct { + serializerChain +} + +var defaultSerializer Serializer = compositeSerializer{ + serializerChain([]Serializer{nilSerializer{}, byteSliceSerializer{}, jsonSerializer{}}), +} diff --git a/vendor/github.com/nexus-rpc/sdk-go/nexus/server.go b/vendor/github.com/nexus-rpc/sdk-go/nexus/server.go new file mode 100644 index 00000000000..34c8c7bd1bb --- /dev/null +++ b/vendor/github.com/nexus-rpc/sdk-go/nexus/server.go @@ -0,0 +1,526 @@ +package nexus + +import ( + "bytes" + "context" + "encoding/json" + "errors" + "fmt" + "io" + "log/slog" + "maps" + "net/http" + "net/url" + "strconv" + "strings" + "time" +) + +// An HandlerStartOperationResult is the return type from the [Handler] StartOperation and [Operation] Start methods. It +// has two implementations: [HandlerStartOperationResultSync] and [HandlerStartOperationResultAsync]. +type HandlerStartOperationResult[T any] interface { + applyToHTTPResponse(http.ResponseWriter, *httpHandler) +} + +// HandlerStartOperationResultSync indicates that an operation completed successfully. +type HandlerStartOperationResultSync[T any] struct { + Value T +} + +func (r *HandlerStartOperationResultSync[T]) applyToHTTPResponse(writer http.ResponseWriter, handler *httpHandler) { + handler.writeResult(writer, r.Value) +} + +// HandlerStartOperationResultAsync indicates that an operation has been accepted and will complete asynchronously. +type HandlerStartOperationResultAsync struct { + OperationID string + Links []Link +} + +func (r *HandlerStartOperationResultAsync) applyToHTTPResponse(writer http.ResponseWriter, handler *httpHandler) { + info := OperationInfo{ + ID: r.OperationID, + State: OperationStateRunning, + } + bytes, err := json.Marshal(info) + if err != nil { + handler.logger.Error("failed to serialize operation info", "error", err) + writer.WriteHeader(http.StatusInternalServerError) + return + } + + if err := addLinksToHTTPHeader(r.Links, writer.Header()); err != nil { + handler.logger.Error("failed to serialize links into header", "error", err) + // clear any previous links already written to the header + writer.Header().Del(headerLink) + writer.WriteHeader(http.StatusInternalServerError) + return + } + + writer.Header().Set("Content-Type", contentTypeJSON) + writer.WriteHeader(http.StatusCreated) + + if _, err := writer.Write(bytes); err != nil { + handler.logger.Error("failed to write response body", "error", err) + } +} + +// A Handler must implement all of the Nexus service endpoints as defined in the [Nexus HTTP API]. +// +// Handler implementations must embed the [UnimplementedHandler]. +// +// All Handler methods can return a [HandlerError] to fail requests with a custom [HandlerErrorType] and structured [Failure]. +// Arbitrary errors from handler methods are turned into [HandlerErrorTypeInternal],their details are logged and hidden +// from the caller. +// +// [Nexus HTTP API]: https://github.com/nexus-rpc/api +type Handler interface { + // StartOperation handles requests for starting an operation. Return [HandlerStartOperationResultSync] to + // respond successfully - inline, or [HandlerStartOperationResultAsync] to indicate that an asynchronous + // operation was started. Return an [UnsuccessfulOperationError] to indicate that an operation completed as + // failed or canceled. + StartOperation(ctx context.Context, service, operation string, input *LazyValue, options StartOperationOptions) (HandlerStartOperationResult[any], error) + // GetOperationResult handles requests to get the result of an asynchronous operation. Return non error result + // to respond successfully - inline, or error with [ErrOperationStillRunning] to indicate that an asynchronous + // operation is still running. Return an [UnsuccessfulOperationError] to indicate that an operation completed as + // failed or canceled. + // + // When [GetOperationResultOptions.Wait] is greater than zero, this request should be treated as a long poll. + // Long poll requests have a server side timeout, configurable via [HandlerOptions.GetResultTimeout], and exposed + // via context deadline. The context deadline is decoupled from the application level Wait duration. + // + // It is the implementor's responsiblity to respect the client's wait duration and return in a timely fashion. + // Consider using a derived context that enforces the wait timeout when implementing this method and return + // [ErrOperationStillRunning] when that context expires as shown in the example. + GetOperationResult(ctx context.Context, service, operation, operationID string, options GetOperationResultOptions) (any, error) + // GetOperationInfo handles requests to get information about an asynchronous operation. + GetOperationInfo(ctx context.Context, service, operation, operationID string, options GetOperationInfoOptions) (*OperationInfo, error) + // CancelOperation handles requests to cancel an asynchronous operation. + // Cancelation in Nexus is: + // 1. asynchronous - returning from this method only ensures that cancelation is delivered, it may later be + // ignored by the underlying operation implemention. + // 2. idempotent - implementors should ignore duplicate cancelations for the same operation. + CancelOperation(ctx context.Context, service, operation, operationID string, options CancelOperationOptions) error + mustEmbedUnimplementedHandler() +} + +type HandlerErrorType string + +const ( + // The server cannot or will not process the request due to an apparent client error. + HandlerErrorTypeBadRequest HandlerErrorType = "BAD_REQUEST" + // The client did not supply valid authentication credentials for this request. + HandlerErrorTypeUnauthenticated HandlerErrorType = "UNAUTHENTICATED" + // The caller does not have permission to execute the specified operation. + HandlerErrorTypeUnauthorized HandlerErrorType = "UNAUTHORIZED" + // The requested resource could not be found but may be available in the future. Subsequent requests by the client + // are permissible. + HandlerErrorTypeNotFound HandlerErrorType = "NOT_FOUND" + // Some resource has been exhausted, perhaps a per-user quota, or perhaps the entire file system is out of space. + HandlerErrorTypeResourceExhausted HandlerErrorType = "RESOURCE_EXHAUSTED" + // An internal error occured. + HandlerErrorTypeInternal HandlerErrorType = "INTERNAL" + // The server either does not recognize the request method, or it lacks the ability to fulfill the request. + HandlerErrorTypeNotImplemented HandlerErrorType = "NOT_IMPLEMENTED" + // The service is currently unavailable. + HandlerErrorTypeUnavailable HandlerErrorType = "UNAVAILABLE" + // Used by gateways to report that a request to a downstream server has timed out. + HandlerErrorTypeDownstreamTimeout HandlerErrorType = "DOWNSTREAM_TIMEOUT" +) + +// HandlerError is a special error that can be returned from [Handler] methods for failing a request with a custom +// status code and failure message. +type HandlerError struct { + // Defaults to HandlerErrorTypeInternal + Type HandlerErrorType + // Failure to report back in the response. Optional. + Failure *Failure +} + +// Error implements the error interface. +func (e *HandlerError) Error() string { + typ := e.Type + if len(typ) == 0 { + typ = HandlerErrorTypeInternal + } + if e.Failure != nil { + return fmt.Sprintf("handler error (%s): %s", typ, e.Failure.Message) + } + return fmt.Sprintf("handler error (%s)", typ) +} + +// HandlerErrorf creates a [HandlerError] with the given type and a formatted failure message. +func HandlerErrorf(typ HandlerErrorType, format string, args ...any) *HandlerError { + return &HandlerError{ + Type: typ, + Failure: &Failure{ + Message: fmt.Sprintf(format, args...), + }, + } +} + +type baseHTTPHandler struct { + logger *slog.Logger +} + +type httpHandler struct { + baseHTTPHandler + options HandlerOptions +} + +func (h *httpHandler) writeResult(writer http.ResponseWriter, result any) { + var reader *Reader + if r, ok := result.(*Reader); ok { + // Close the request body in case we error before sending the HTTP request (which may double close but + // that's fine since we ignore the error). + defer r.Close() + reader = r + } else { + content, ok := result.(*Content) + if !ok { + var err error + content, err = h.options.Serializer.Serialize(result) + if err != nil { + h.writeFailure(writer, fmt.Errorf("failed to serialize handler result: %w", err)) + return + } + } + header := maps.Clone(content.Header) + header["length"] = strconv.Itoa(len(content.Data)) + + reader = &Reader{ + io.NopCloser(bytes.NewReader(content.Data)), + header, + } + } + + header := writer.Header() + addContentHeaderToHTTPHeader(reader.Header, header) + if reader.ReadCloser == nil { + return + } + if _, err := io.Copy(writer, reader); err != nil { + h.logger.Error("failed to write response body", "error", err) + } +} + +func (h *baseHTTPHandler) writeFailure(writer http.ResponseWriter, err error) { + var failure *Failure + var unsuccessfulError *UnsuccessfulOperationError + var handlerError *HandlerError + var operationState OperationState + statusCode := http.StatusInternalServerError + + if errors.As(err, &unsuccessfulError) { + operationState = unsuccessfulError.State + failure = &unsuccessfulError.Failure + statusCode = statusOperationFailed + + if operationState == OperationStateFailed || operationState == OperationStateCanceled { + writer.Header().Set(headerOperationState, string(operationState)) + } else { + h.logger.Error("unexpected operation state", "state", operationState) + writer.WriteHeader(http.StatusInternalServerError) + return + } + } else if errors.As(err, &handlerError) { + failure = handlerError.Failure + switch handlerError.Type { + case HandlerErrorTypeBadRequest: + statusCode = http.StatusBadRequest + case HandlerErrorTypeUnauthenticated: + statusCode = http.StatusUnauthorized + case HandlerErrorTypeUnauthorized: + statusCode = http.StatusForbidden + case HandlerErrorTypeNotFound: + statusCode = http.StatusNotFound + case HandlerErrorTypeResourceExhausted: + statusCode = http.StatusTooManyRequests + case HandlerErrorTypeInternal: + statusCode = http.StatusInternalServerError + case HandlerErrorTypeNotImplemented: + statusCode = http.StatusNotImplemented + case HandlerErrorTypeUnavailable: + statusCode = http.StatusServiceUnavailable + case HandlerErrorTypeDownstreamTimeout: + statusCode = StatusDownstreamTimeout + default: + h.logger.Error("unexpected handler error type", "type", handlerError.Type) + } + } else { + failure = &Failure{ + Message: "internal server error", + } + h.logger.Error("handler failed", "error", err) + } + + var bytes []byte + if failure != nil { + bytes, err = json.Marshal(failure) + if err != nil { + h.logger.Error("failed to marshal failure", "error", err) + writer.WriteHeader(http.StatusInternalServerError) + return + } + writer.Header().Set("Content-Type", contentTypeJSON) + } + + writer.WriteHeader(statusCode) + + if _, err := writer.Write(bytes); err != nil { + h.logger.Error("failed to write response body", "error", err) + } +} + +func (h *httpHandler) startOperation(service, operation string, writer http.ResponseWriter, request *http.Request) { + links, err := getLinksFromHeader(request.Header) + if err != nil { + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "invalid %q header", headerLink)) + return + } + options := StartOperationOptions{ + RequestID: request.Header.Get(headerRequestID), + CallbackURL: request.URL.Query().Get(queryCallbackURL), + CallbackHeader: prefixStrippedHTTPHeaderToNexusHeader(request.Header, "nexus-callback-"), + Header: httpHeaderToNexusHeader(request.Header, "content-", "nexus-callback-"), + Links: links, + } + value := &LazyValue{ + serializer: h.options.Serializer, + Reader: &Reader{ + request.Body, + prefixStrippedHTTPHeaderToNexusHeader(request.Header, "content-"), + }, + } + + ctx, cancel, ok := h.contextWithTimeoutFromHTTPRequest(writer, request) + if !ok { + return + } + defer cancel() + + response, err := h.options.Handler.StartOperation(ctx, service, operation, value, options) + if err != nil { + h.writeFailure(writer, err) + } else { + response.applyToHTTPResponse(writer, h) + } +} + +func (h *httpHandler) getOperationResult(service, operation, operationID string, writer http.ResponseWriter, request *http.Request) { + options := GetOperationResultOptions{Header: httpHeaderToNexusHeader(request.Header)} + + // If both Request-Timeout http header and wait query string are set, the minimum of the Request-Timeout header + // and h.options.GetResultTimeout will be used. + ctx := request.Context() + requestTimeout, ok := h.parseRequestTimeoutHeader(writer, request) + if !ok { + return + } + waitStr := request.URL.Query().Get(queryWait) + if waitStr != "" { + waitDuration, err := time.ParseDuration(waitStr) + if err != nil { + h.logger.Warn("invalid wait duration query parameter", "wait", waitStr) + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "invalid wait query parameter")) + return + } + options.Wait = waitDuration + if requestTimeout > 0 { + requestTimeout = min(requestTimeout, h.options.GetResultTimeout) + } else { + requestTimeout = h.options.GetResultTimeout + } + } + if requestTimeout > 0 { + var cancel context.CancelFunc + ctx, cancel = context.WithTimeout(request.Context(), requestTimeout) + defer cancel() + } + + result, err := h.options.Handler.GetOperationResult(ctx, service, operation, operationID, options) + if err != nil { + if options.Wait > 0 && ctx.Err() != nil { + writer.WriteHeader(http.StatusRequestTimeout) + } else if errors.Is(err, ErrOperationStillRunning) { + writer.WriteHeader(statusOperationRunning) + } else { + h.writeFailure(writer, err) + } + return + } + h.writeResult(writer, result) +} + +func (h *httpHandler) getOperationInfo(service, operation, operationID string, writer http.ResponseWriter, request *http.Request) { + options := GetOperationInfoOptions{Header: httpHeaderToNexusHeader(request.Header)} + + ctx, cancel, ok := h.contextWithTimeoutFromHTTPRequest(writer, request) + if !ok { + return + } + defer cancel() + + info, err := h.options.Handler.GetOperationInfo(ctx, service, operation, operationID, options) + if err != nil { + h.writeFailure(writer, err) + return + } + + bytes, err := json.Marshal(info) + if err != nil { + h.writeFailure(writer, fmt.Errorf("failed to marshal operation info: %w", err)) + return + } + writer.Header().Set("Content-Type", contentTypeJSON) + if _, err := writer.Write(bytes); err != nil { + h.logger.Error("failed to write response body", "error", err) + } +} + +func (h *httpHandler) cancelOperation(service, operation, operationID string, writer http.ResponseWriter, request *http.Request) { + options := CancelOperationOptions{Header: httpHeaderToNexusHeader(request.Header)} + + ctx, cancel, ok := h.contextWithTimeoutFromHTTPRequest(writer, request) + if !ok { + return + } + defer cancel() + + if err := h.options.Handler.CancelOperation(ctx, service, operation, operationID, options); err != nil { + h.writeFailure(writer, err) + return + } + + writer.WriteHeader(http.StatusAccepted) +} + +// parseRequestTimeoutHeader checks if the Request-Timeout HTTP header is set and returns the parsed duration if so. +// Returns (0, true) if unset. Returns ({parsedDuration}, true) if set. If set and there is an error parsing the +// duration, it writes a failure response and returns (0, false). +func (h *httpHandler) parseRequestTimeoutHeader(writer http.ResponseWriter, request *http.Request) (time.Duration, bool) { + timeoutStr := request.Header.Get(HeaderRequestTimeout) + if timeoutStr != "" { + timeoutDuration, err := time.ParseDuration(timeoutStr) + if err != nil { + h.logger.Warn("invalid request timeout header", "timeout", timeoutStr) + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "invalid request timeout header")) + return 0, false + } + return timeoutDuration, true + } + return 0, true +} + +// contextWithTimeoutFromHTTPRequest extracts the context from the HTTP request and applies the timeout indicated by +// the Request-Timeout header, if set. +func (h *httpHandler) contextWithTimeoutFromHTTPRequest(writer http.ResponseWriter, request *http.Request) (context.Context, context.CancelFunc, bool) { + requestTimeout, ok := h.parseRequestTimeoutHeader(writer, request) + if !ok { + return nil, nil, false + } + if requestTimeout > 0 { + ctx, cancel := context.WithTimeout(request.Context(), requestTimeout) + return ctx, cancel, true + } + return request.Context(), func() {}, true +} + +// HandlerOptions are options for [NewHTTPHandler]. +type HandlerOptions struct { + // Handler for handling service requests. + Handler Handler + // A stuctured logger. + // Defaults to slog.Default(). + Logger *slog.Logger + // Max duration to allow waiting for a single get result request. + // Enforced if provided for requests with the wait query parameter set. + // + // Defaults to one minute. + GetResultTimeout time.Duration + // A [Serializer] to customize handler serialization behavior. + // By default the handler handles, JSONables, byte slices, and nil. + Serializer Serializer +} + +func (h *httpHandler) handleRequest(writer http.ResponseWriter, request *http.Request) { + parts := strings.Split(request.URL.EscapedPath(), "/") + // First part is empty (due to leading /) + if len(parts) < 3 { + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeNotFound, "not found")) + return + } + service, err := url.PathUnescape(parts[1]) + if err != nil { + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "failed to parse URL path")) + return + } + operation, err := url.PathUnescape(parts[2]) + if err != nil { + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "failed to parse URL path")) + return + } + var operationID string + if len(parts) > 3 { + operationID, err = url.PathUnescape(parts[3]) + if err != nil { + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "failed to parse URL path")) + return + } + } + + switch len(parts) { + case 3: // /{service}/{operation} + if request.Method != "POST" { + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "invalid request method: expected POST, got %q", request.Method)) + return + } + h.startOperation(service, operation, writer, request) + case 4: // /{service}/{operation}/{operation_id} + if request.Method != "GET" { + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "invalid request method: expected GET, got %q", request.Method)) + return + } + h.getOperationInfo(service, operation, operationID, writer, request) + case 5: + switch parts[4] { + case "result": // /{service}/{operation}/{operation_id}/result + if request.Method != "GET" { + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "invalid request method: expected GET, got %q", request.Method)) + return + } + h.getOperationResult(service, operation, operationID, writer, request) + case "cancel": // /{service}/{operation}/{operation_id}/cancel + if request.Method != "POST" { + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeBadRequest, "invalid request method: expected POST, got %q", request.Method)) + return + } + h.cancelOperation(service, operation, operationID, writer, request) + default: + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeNotFound, "not found")) + } + default: + h.writeFailure(writer, HandlerErrorf(HandlerErrorTypeNotFound, "not found")) + } +} + +// NewHTTPHandler constructs an [http.Handler] from given options for handling Nexus service requests. +func NewHTTPHandler(options HandlerOptions) http.Handler { + if options.Logger == nil { + options.Logger = slog.Default() + } + if options.GetResultTimeout == 0 { + options.GetResultTimeout = time.Minute + } + if options.Serializer == nil { + options.Serializer = defaultSerializer + } + handler := &httpHandler{ + baseHTTPHandler: baseHTTPHandler{ + logger: options.Logger, + }, + options: options, + } + + return http.HandlerFunc(handler.handleRequest) +} diff --git a/vendor/github.com/nexus-rpc/sdk-go/nexus/unimplemented_handler.go b/vendor/github.com/nexus-rpc/sdk-go/nexus/unimplemented_handler.go new file mode 100644 index 00000000000..9145226469a --- /dev/null +++ b/vendor/github.com/nexus-rpc/sdk-go/nexus/unimplemented_handler.go @@ -0,0 +1,62 @@ +package nexus + +import ( + "context" +) + +// UnimplementedHandler must be embedded into any [Handler] implementation for future compatibility. +// It implements all methods on the [Handler] interface, returning unimplemented errors if they are not implemented by +// the embedding type. +type UnimplementedHandler struct{} + +func (h UnimplementedHandler) mustEmbedUnimplementedHandler() {} + +// StartOperation implements the Handler interface. +func (h UnimplementedHandler) StartOperation(ctx context.Context, service, operation string, input *LazyValue, options StartOperationOptions) (HandlerStartOperationResult[any], error) { + return nil, &HandlerError{HandlerErrorTypeNotImplemented, &Failure{Message: "not implemented"}} +} + +// GetOperationResult implements the Handler interface. +func (h UnimplementedHandler) GetOperationResult(ctx context.Context, service, operation, operationID string, options GetOperationResultOptions) (any, error) { + return nil, &HandlerError{HandlerErrorTypeNotImplemented, &Failure{Message: "not implemented"}} +} + +// GetOperationInfo implements the Handler interface. +func (h UnimplementedHandler) GetOperationInfo(ctx context.Context, service, operation, operationID string, options GetOperationInfoOptions) (*OperationInfo, error) { + return nil, &HandlerError{HandlerErrorTypeNotImplemented, &Failure{Message: "not implemented"}} +} + +// CancelOperation implements the Handler interface. +func (h UnimplementedHandler) CancelOperation(ctx context.Context, service, operation, operationID string, options CancelOperationOptions) error { + return &HandlerError{HandlerErrorTypeNotImplemented, &Failure{Message: "not implemented"}} +} + +// UnimplementedOperation must be embedded into any [Operation] implementation for future compatibility. +// It implements all methods on the [Operation] interface except for `Name`, returning unimplemented errors if they are +// not implemented by the embedding type. +type UnimplementedOperation[I, O any] struct{} + +func (*UnimplementedOperation[I, O]) inferType(I, O) {} //nolint:unused + +func (*UnimplementedOperation[I, O]) mustEmbedUnimplementedOperation() {} + +// Cancel implements Operation. +func (*UnimplementedOperation[I, O]) Cancel(context.Context, string, CancelOperationOptions) error { + return HandlerErrorf(HandlerErrorTypeNotImplemented, "not implemented") +} + +// GetInfo implements Operation. +func (*UnimplementedOperation[I, O]) GetInfo(context.Context, string, GetOperationInfoOptions) (*OperationInfo, error) { + return nil, HandlerErrorf(HandlerErrorTypeNotImplemented, "not implemented") +} + +// GetResult implements Operation. +func (*UnimplementedOperation[I, O]) GetResult(context.Context, string, GetOperationResultOptions) (O, error) { + var empty O + return empty, HandlerErrorf(HandlerErrorTypeNotImplemented, "not implemented") +} + +// Start implements Operation. +func (h *UnimplementedOperation[I, O]) Start(ctx context.Context, input I, options StartOperationOptions) (HandlerStartOperationResult[O], error) { + return nil, HandlerErrorf(HandlerErrorTypeNotImplemented, "not implemented") +} diff --git a/vendor/github.com/pborman/uuid/.travis.yml b/vendor/github.com/pborman/uuid/.travis.yml new file mode 100644 index 00000000000..3deb4a12430 --- /dev/null +++ b/vendor/github.com/pborman/uuid/.travis.yml @@ -0,0 +1,10 @@ +language: go + +go: + - "1.9" + - "1.10" + - "1.11" + - tip + +script: + - go test -v ./... diff --git a/vendor/github.com/pborman/uuid/CONTRIBUTING.md b/vendor/github.com/pborman/uuid/CONTRIBUTING.md new file mode 100644 index 00000000000..04fdf09f136 --- /dev/null +++ b/vendor/github.com/pborman/uuid/CONTRIBUTING.md @@ -0,0 +1,10 @@ +# How to contribute + +We definitely welcome patches and contribution to this project! + +### Legal requirements + +In order to protect both you and ourselves, you will need to sign the +[Contributor License Agreement](https://cla.developers.google.com/clas). + +You may have already signed it for other Google projects. diff --git a/vendor/github.com/pborman/uuid/CONTRIBUTORS b/vendor/github.com/pborman/uuid/CONTRIBUTORS new file mode 100644 index 00000000000..b382a04eda9 --- /dev/null +++ b/vendor/github.com/pborman/uuid/CONTRIBUTORS @@ -0,0 +1 @@ +Paul Borman diff --git a/vendor/github.com/pborman/uuid/LICENSE b/vendor/github.com/pborman/uuid/LICENSE new file mode 100644 index 00000000000..5dc68268d90 --- /dev/null +++ b/vendor/github.com/pborman/uuid/LICENSE @@ -0,0 +1,27 @@ +Copyright (c) 2009,2014 Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/pborman/uuid/README.md b/vendor/github.com/pborman/uuid/README.md new file mode 100644 index 00000000000..810ad40dc93 --- /dev/null +++ b/vendor/github.com/pborman/uuid/README.md @@ -0,0 +1,15 @@ +This project was automatically exported from code.google.com/p/go-uuid + +# uuid ![build status](https://travis-ci.org/pborman/uuid.svg?branch=master) +The uuid package generates and inspects UUIDs based on [RFC 4122](http://tools.ietf.org/html/rfc4122) and DCE 1.1: Authentication and Security Services. + +This package now leverages the github.com/google/uuid package (which is based off an earlier version of this package). + +###### Install +`go get github.com/pborman/uuid` + +###### Documentation +[![GoDoc](https://godoc.org/github.com/pborman/uuid?status.svg)](http://godoc.org/github.com/pborman/uuid) + +Full `go doc` style documentation for the package can be viewed online without installing this package by using the GoDoc site here: +http://godoc.org/github.com/pborman/uuid diff --git a/vendor/github.com/pborman/uuid/dce.go b/vendor/github.com/pborman/uuid/dce.go new file mode 100644 index 00000000000..50a0f2d0992 --- /dev/null +++ b/vendor/github.com/pborman/uuid/dce.go @@ -0,0 +1,84 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "encoding/binary" + "fmt" + "os" +) + +// A Domain represents a Version 2 domain +type Domain byte + +// Domain constants for DCE Security (Version 2) UUIDs. +const ( + Person = Domain(0) + Group = Domain(1) + Org = Domain(2) +) + +// NewDCESecurity returns a DCE Security (Version 2) UUID. +// +// The domain should be one of Person, Group or Org. +// On a POSIX system the id should be the users UID for the Person +// domain and the users GID for the Group. The meaning of id for +// the domain Org or on non-POSIX systems is site defined. +// +// For a given domain/id pair the same token may be returned for up to +// 7 minutes and 10 seconds. +func NewDCESecurity(domain Domain, id uint32) UUID { + uuid := NewUUID() + if uuid != nil { + uuid[6] = (uuid[6] & 0x0f) | 0x20 // Version 2 + uuid[9] = byte(domain) + binary.BigEndian.PutUint32(uuid[0:], id) + } + return uuid +} + +// NewDCEPerson returns a DCE Security (Version 2) UUID in the person +// domain with the id returned by os.Getuid. +// +// NewDCEPerson(Person, uint32(os.Getuid())) +func NewDCEPerson() UUID { + return NewDCESecurity(Person, uint32(os.Getuid())) +} + +// NewDCEGroup returns a DCE Security (Version 2) UUID in the group +// domain with the id returned by os.Getgid. +// +// NewDCEGroup(Group, uint32(os.Getgid())) +func NewDCEGroup() UUID { + return NewDCESecurity(Group, uint32(os.Getgid())) +} + +// Domain returns the domain for a Version 2 UUID or false. +func (uuid UUID) Domain() (Domain, bool) { + if v, _ := uuid.Version(); v != 2 { + return 0, false + } + return Domain(uuid[9]), true +} + +// Id returns the id for a Version 2 UUID or false. +func (uuid UUID) Id() (uint32, bool) { + if v, _ := uuid.Version(); v != 2 { + return 0, false + } + return binary.BigEndian.Uint32(uuid[0:4]), true +} + +func (d Domain) String() string { + switch d { + case Person: + return "Person" + case Group: + return "Group" + case Org: + return "Org" + } + return fmt.Sprintf("Domain%d", int(d)) +} diff --git a/vendor/github.com/pborman/uuid/doc.go b/vendor/github.com/pborman/uuid/doc.go new file mode 100644 index 00000000000..727d7616745 --- /dev/null +++ b/vendor/github.com/pborman/uuid/doc.go @@ -0,0 +1,13 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// The uuid package generates and inspects UUIDs. +// +// UUIDs are based on RFC 4122 and DCE 1.1: Authentication and Security +// Services. +// +// This package is a partial wrapper around the github.com/google/uuid package. +// This package represents a UUID as []byte while github.com/google/uuid +// represents a UUID as [16]byte. +package uuid diff --git a/vendor/github.com/pborman/uuid/hash.go b/vendor/github.com/pborman/uuid/hash.go new file mode 100644 index 00000000000..a0420c1ef3a --- /dev/null +++ b/vendor/github.com/pborman/uuid/hash.go @@ -0,0 +1,53 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "crypto/md5" + "crypto/sha1" + "hash" +) + +// Well known Name Space IDs and UUIDs +var ( + NameSpace_DNS = Parse("6ba7b810-9dad-11d1-80b4-00c04fd430c8") + NameSpace_URL = Parse("6ba7b811-9dad-11d1-80b4-00c04fd430c8") + NameSpace_OID = Parse("6ba7b812-9dad-11d1-80b4-00c04fd430c8") + NameSpace_X500 = Parse("6ba7b814-9dad-11d1-80b4-00c04fd430c8") + NIL = Parse("00000000-0000-0000-0000-000000000000") +) + +// NewHash returns a new UUID derived from the hash of space concatenated with +// data generated by h. The hash should be at least 16 byte in length. The +// first 16 bytes of the hash are used to form the UUID. The version of the +// UUID will be the lower 4 bits of version. NewHash is used to implement +// NewMD5 and NewSHA1. +func NewHash(h hash.Hash, space UUID, data []byte, version int) UUID { + h.Reset() + h.Write(space) + h.Write([]byte(data)) + s := h.Sum(nil) + uuid := make([]byte, 16) + copy(uuid, s) + uuid[6] = (uuid[6] & 0x0f) | uint8((version&0xf)<<4) + uuid[8] = (uuid[8] & 0x3f) | 0x80 // RFC 4122 variant + return uuid +} + +// NewMD5 returns a new MD5 (Version 3) UUID based on the +// supplied name space and data. +// +// NewHash(md5.New(), space, data, 3) +func NewMD5(space UUID, data []byte) UUID { + return NewHash(md5.New(), space, data, 3) +} + +// NewSHA1 returns a new SHA1 (Version 5) UUID based on the +// supplied name space and data. +// +// NewHash(sha1.New(), space, data, 5) +func NewSHA1(space UUID, data []byte) UUID { + return NewHash(sha1.New(), space, data, 5) +} diff --git a/vendor/github.com/pborman/uuid/marshal.go b/vendor/github.com/pborman/uuid/marshal.go new file mode 100644 index 00000000000..35b89352ad7 --- /dev/null +++ b/vendor/github.com/pborman/uuid/marshal.go @@ -0,0 +1,85 @@ +// Copyright 2016 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "errors" + "fmt" + + guuid "github.com/google/uuid" +) + +// MarshalText implements encoding.TextMarshaler. +func (u UUID) MarshalText() ([]byte, error) { + if len(u) != 16 { + return nil, nil + } + var js [36]byte + encodeHex(js[:], u) + return js[:], nil +} + +// UnmarshalText implements encoding.TextUnmarshaler. +func (u *UUID) UnmarshalText(data []byte) error { + if len(data) == 0 { + return nil + } + id := Parse(string(data)) + if id == nil { + return errors.New("invalid UUID") + } + *u = id + return nil +} + +// MarshalBinary implements encoding.BinaryMarshaler. +func (u UUID) MarshalBinary() ([]byte, error) { + return u[:], nil +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler. +func (u *UUID) UnmarshalBinary(data []byte) error { + if len(data) == 0 { + return nil + } + if len(data) != 16 { + return fmt.Errorf("invalid UUID (got %d bytes)", len(data)) + } + var id [16]byte + copy(id[:], data) + *u = id[:] + return nil +} + +// MarshalText implements encoding.TextMarshaler. +func (u Array) MarshalText() ([]byte, error) { + var js [36]byte + encodeHex(js[:], u[:]) + return js[:], nil +} + +// UnmarshalText implements encoding.TextUnmarshaler. +func (u *Array) UnmarshalText(data []byte) error { + id, err := guuid.ParseBytes(data) + if err != nil { + return err + } + *u = Array(id) + return nil +} + +// MarshalBinary implements encoding.BinaryMarshaler. +func (u Array) MarshalBinary() ([]byte, error) { + return u[:], nil +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler. +func (u *Array) UnmarshalBinary(data []byte) error { + if len(data) != 16 { + return fmt.Errorf("invalid UUID (got %d bytes)", len(data)) + } + copy(u[:], data) + return nil +} diff --git a/vendor/github.com/pborman/uuid/node.go b/vendor/github.com/pborman/uuid/node.go new file mode 100644 index 00000000000..e524e0101b4 --- /dev/null +++ b/vendor/github.com/pborman/uuid/node.go @@ -0,0 +1,50 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + guuid "github.com/google/uuid" +) + +// NodeInterface returns the name of the interface from which the NodeID was +// derived. The interface "user" is returned if the NodeID was set by +// SetNodeID. +func NodeInterface() string { + return guuid.NodeInterface() +} + +// SetNodeInterface selects the hardware address to be used for Version 1 UUIDs. +// If name is "" then the first usable interface found will be used or a random +// Node ID will be generated. If a named interface cannot be found then false +// is returned. +// +// SetNodeInterface never fails when name is "". +func SetNodeInterface(name string) bool { + return guuid.SetNodeInterface(name) +} + +// NodeID returns a slice of a copy of the current Node ID, setting the Node ID +// if not already set. +func NodeID() []byte { + return guuid.NodeID() +} + +// SetNodeID sets the Node ID to be used for Version 1 UUIDs. The first 6 bytes +// of id are used. If id is less than 6 bytes then false is returned and the +// Node ID is not set. +func SetNodeID(id []byte) bool { + return guuid.SetNodeID(id) +} + +// NodeID returns the 6 byte node id encoded in uuid. It returns nil if uuid is +// not valid. The NodeID is only well defined for version 1 and 2 UUIDs. +func (uuid UUID) NodeID() []byte { + if len(uuid) != 16 { + return nil + } + node := make([]byte, 6) + copy(node, uuid[10:]) + return node +} diff --git a/vendor/github.com/pborman/uuid/sql.go b/vendor/github.com/pborman/uuid/sql.go new file mode 100644 index 00000000000..929c3847e2a --- /dev/null +++ b/vendor/github.com/pborman/uuid/sql.go @@ -0,0 +1,68 @@ +// Copyright 2015 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "database/sql/driver" + "errors" + "fmt" +) + +// Scan implements sql.Scanner so UUIDs can be read from databases transparently +// Currently, database types that map to string and []byte are supported. Please +// consult database-specific driver documentation for matching types. +func (uuid *UUID) Scan(src interface{}) error { + switch src.(type) { + case string: + // if an empty UUID comes from a table, we return a null UUID + if src.(string) == "" { + return nil + } + + // see uuid.Parse for required string format + parsed := Parse(src.(string)) + + if parsed == nil { + return errors.New("Scan: invalid UUID format") + } + + *uuid = parsed + case []byte: + b := src.([]byte) + + // if an empty UUID comes from a table, we return a null UUID + if len(b) == 0 { + return nil + } + + // assumes a simple slice of bytes if 16 bytes + // otherwise attempts to parse + if len(b) == 16 { + parsed := make([]byte, 16) + copy(parsed, b) + *uuid = UUID(parsed) + } else { + u := Parse(string(b)) + + if u == nil { + return errors.New("Scan: invalid UUID format") + } + + *uuid = u + } + + default: + return fmt.Errorf("Scan: unable to scan type %T into UUID", src) + } + + return nil +} + +// Value implements sql.Valuer so that UUIDs can be written to databases +// transparently. Currently, UUIDs map to strings. Please consult +// database-specific driver documentation for matching types. +func (uuid UUID) Value() (driver.Value, error) { + return uuid.String(), nil +} diff --git a/vendor/github.com/pborman/uuid/time.go b/vendor/github.com/pborman/uuid/time.go new file mode 100644 index 00000000000..7286824d899 --- /dev/null +++ b/vendor/github.com/pborman/uuid/time.go @@ -0,0 +1,57 @@ +// Copyright 2014 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "encoding/binary" + + guuid "github.com/google/uuid" +) + +// A Time represents a time as the number of 100's of nanoseconds since 15 Oct +// 1582. +type Time = guuid.Time + +// GetTime returns the current Time (100s of nanoseconds since 15 Oct 1582) and +// clock sequence as well as adjusting the clock sequence as needed. An error +// is returned if the current time cannot be determined. +func GetTime() (Time, uint16, error) { return guuid.GetTime() } + +// ClockSequence returns the current clock sequence, generating one if not +// already set. The clock sequence is only used for Version 1 UUIDs. +// +// The uuid package does not use global static storage for the clock sequence or +// the last time a UUID was generated. Unless SetClockSequence a new random +// clock sequence is generated the first time a clock sequence is requested by +// ClockSequence, GetTime, or NewUUID. (section 4.2.1.1) sequence is generated +// for +func ClockSequence() int { return guuid.ClockSequence() } + +// SetClockSequence sets the clock sequence to the lower 14 bits of seq. Setting to +// -1 causes a new sequence to be generated. +func SetClockSequence(seq int) { guuid.SetClockSequence(seq) } + +// Time returns the time in 100s of nanoseconds since 15 Oct 1582 encoded in +// uuid. It returns false if uuid is not valid. The time is only well defined +// for version 1 and 2 UUIDs. +func (uuid UUID) Time() (Time, bool) { + if len(uuid) != 16 { + return 0, false + } + time := int64(binary.BigEndian.Uint32(uuid[0:4])) + time |= int64(binary.BigEndian.Uint16(uuid[4:6])) << 32 + time |= int64(binary.BigEndian.Uint16(uuid[6:8])&0xfff) << 48 + return Time(time), true +} + +// ClockSequence returns the clock sequence encoded in uuid. It returns false +// if uuid is not valid. The clock sequence is only well defined for version 1 +// and 2 UUIDs. +func (uuid UUID) ClockSequence() (int, bool) { + if len(uuid) != 16 { + return 0, false + } + return int(binary.BigEndian.Uint16(uuid[8:10])) & 0x3fff, true +} diff --git a/vendor/github.com/pborman/uuid/util.go b/vendor/github.com/pborman/uuid/util.go new file mode 100644 index 00000000000..255b5e24859 --- /dev/null +++ b/vendor/github.com/pborman/uuid/util.go @@ -0,0 +1,32 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +// xvalues returns the value of a byte as a hexadecimal digit or 255. +var xvalues = [256]byte{ + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 255, 255, 255, 255, 255, 255, + 255, 10, 11, 12, 13, 14, 15, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 10, 11, 12, 13, 14, 15, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, +} + +// xtob converts the the first two hex bytes of x into a byte. +func xtob(x string) (byte, bool) { + b1 := xvalues[x[0]] + b2 := xvalues[x[1]] + return (b1 << 4) | b2, b1 != 255 && b2 != 255 +} diff --git a/vendor/github.com/pborman/uuid/uuid.go b/vendor/github.com/pborman/uuid/uuid.go new file mode 100644 index 00000000000..33700042072 --- /dev/null +++ b/vendor/github.com/pborman/uuid/uuid.go @@ -0,0 +1,162 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "bytes" + "crypto/rand" + "encoding/hex" + "io" + + guuid "github.com/google/uuid" +) + +// Array is a pass-by-value UUID that can be used as an effecient key in a map. +type Array [16]byte + +// UUID converts uuid into a slice. +func (uuid Array) UUID() UUID { + return uuid[:] +} + +// String returns the string representation of uuid, +// xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx. +func (uuid Array) String() string { + return guuid.UUID(uuid).String() +} + +// A UUID is a 128 bit (16 byte) Universal Unique IDentifier as defined in RFC +// 4122. +type UUID []byte + +// A Version represents a UUIDs version. +type Version = guuid.Version + +// A Variant represents a UUIDs variant. +type Variant = guuid.Variant + +// Constants returned by Variant. +const ( + Invalid = guuid.Invalid // Invalid UUID + RFC4122 = guuid.RFC4122 // The variant specified in RFC4122 + Reserved = guuid.Reserved // Reserved, NCS backward compatibility. + Microsoft = guuid.Microsoft // Reserved, Microsoft Corporation backward compatibility. + Future = guuid.Future // Reserved for future definition. +) + +var rander = rand.Reader // random function + +// New returns a new random (version 4) UUID as a string. It is a convenience +// function for NewRandom().String(). +func New() string { + return NewRandom().String() +} + +// Parse decodes s into a UUID or returns nil. See github.com/google/uuid for +// the formats parsed. +func Parse(s string) UUID { + gu, err := guuid.Parse(s) + if err == nil { + return gu[:] + } + return nil +} + +// ParseBytes is like Parse, except it parses a byte slice instead of a string. +func ParseBytes(b []byte) (UUID, error) { + gu, err := guuid.ParseBytes(b) + if err == nil { + return gu[:], nil + } + return nil, err +} + +// Equal returns true if uuid1 and uuid2 are equal. +func Equal(uuid1, uuid2 UUID) bool { + return bytes.Equal(uuid1, uuid2) +} + +// Array returns an array representation of uuid that can be used as a map key. +// Array panics if uuid is not valid. +func (uuid UUID) Array() Array { + if len(uuid) != 16 { + panic("invalid uuid") + } + var a Array + copy(a[:], uuid) + return a +} + +// String returns the string form of uuid, xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx +// , or "" if uuid is invalid. +func (uuid UUID) String() string { + if len(uuid) != 16 { + return "" + } + var buf [36]byte + encodeHex(buf[:], uuid) + return string(buf[:]) +} + +// URN returns the RFC 2141 URN form of uuid, +// urn:uuid:xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx, or "" if uuid is invalid. +func (uuid UUID) URN() string { + if len(uuid) != 16 { + return "" + } + var buf [36 + 9]byte + copy(buf[:], "urn:uuid:") + encodeHex(buf[9:], uuid) + return string(buf[:]) +} + +func encodeHex(dst []byte, uuid UUID) { + hex.Encode(dst[:], uuid[:4]) + dst[8] = '-' + hex.Encode(dst[9:13], uuid[4:6]) + dst[13] = '-' + hex.Encode(dst[14:18], uuid[6:8]) + dst[18] = '-' + hex.Encode(dst[19:23], uuid[8:10]) + dst[23] = '-' + hex.Encode(dst[24:], uuid[10:]) +} + +// Variant returns the variant encoded in uuid. It returns Invalid if +// uuid is invalid. +func (uuid UUID) Variant() Variant { + if len(uuid) != 16 { + return Invalid + } + switch { + case (uuid[8] & 0xc0) == 0x80: + return RFC4122 + case (uuid[8] & 0xe0) == 0xc0: + return Microsoft + case (uuid[8] & 0xe0) == 0xe0: + return Future + default: + return Reserved + } +} + +// Version returns the version of uuid. It returns false if uuid is not +// valid. +func (uuid UUID) Version() (Version, bool) { + if len(uuid) != 16 { + return 0, false + } + return Version(uuid[6] >> 4), true +} + +// SetRand sets the random number generator to r, which implements io.Reader. +// If r.Read returns an error when the package requests random data then +// a panic will be issued. +// +// Calling SetRand with nil sets the random number generator to the default +// generator. +func SetRand(r io.Reader) { + guuid.SetRand(r) +} diff --git a/vendor/github.com/pborman/uuid/version1.go b/vendor/github.com/pborman/uuid/version1.go new file mode 100644 index 00000000000..7af948da793 --- /dev/null +++ b/vendor/github.com/pborman/uuid/version1.go @@ -0,0 +1,23 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + guuid "github.com/google/uuid" +) + +// NewUUID returns a Version 1 UUID based on the current NodeID and clock +// sequence, and the current time. If the NodeID has not been set by SetNodeID +// or SetNodeInterface then it will be set automatically. If the NodeID cannot +// be set NewUUID returns nil. If clock sequence has not been set by +// SetClockSequence then it will be set automatically. If GetTime fails to +// return the current NewUUID returns nil. +func NewUUID() UUID { + gu, err := guuid.NewUUID() + if err == nil { + return UUID(gu[:]) + } + return nil +} diff --git a/vendor/github.com/pborman/uuid/version4.go b/vendor/github.com/pborman/uuid/version4.go new file mode 100644 index 00000000000..767dd0c3aaf --- /dev/null +++ b/vendor/github.com/pborman/uuid/version4.go @@ -0,0 +1,26 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import guuid "github.com/google/uuid" + +// NewRandom returns a Random (Version 4) UUID or panics. +// +// The strength of the UUIDs is based on the strength of the crypto/rand +// package. +// +// A note about uniqueness derived from the UUID Wikipedia entry: +// +// Randomly generated UUIDs have 122 random bits. One's annual risk of being +// hit by a meteorite is estimated to be one chance in 17 billion, that +// means the probability is about 0.00000000006 (6 × 10−11), +// equivalent to the odds of creating a few tens of trillions of UUIDs in a +// year and having one duplicate. +func NewRandom() UUID { + if gu, err := guuid.NewRandom(); err == nil { + return UUID(gu[:]) + } + return nil +} diff --git a/vendor/github.com/robfig/cron/.gitignore b/vendor/github.com/robfig/cron/.gitignore new file mode 100644 index 00000000000..00268614f04 --- /dev/null +++ b/vendor/github.com/robfig/cron/.gitignore @@ -0,0 +1,22 @@ +# Compiled Object files, Static and Dynamic libs (Shared Objects) +*.o +*.a +*.so + +# Folders +_obj +_test + +# Architecture specific extensions/prefixes +*.[568vq] +[568vq].out + +*.cgo1.go +*.cgo2.c +_cgo_defun.c +_cgo_gotypes.go +_cgo_export.* + +_testmain.go + +*.exe diff --git a/vendor/github.com/robfig/cron/.travis.yml b/vendor/github.com/robfig/cron/.travis.yml new file mode 100644 index 00000000000..4f2ee4d9733 --- /dev/null +++ b/vendor/github.com/robfig/cron/.travis.yml @@ -0,0 +1 @@ +language: go diff --git a/vendor/github.com/robfig/cron/LICENSE b/vendor/github.com/robfig/cron/LICENSE new file mode 100644 index 00000000000..3a0f627ffeb --- /dev/null +++ b/vendor/github.com/robfig/cron/LICENSE @@ -0,0 +1,21 @@ +Copyright (C) 2012 Rob Figueiredo +All Rights Reserved. + +MIT LICENSE + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of +the Software, and to permit persons to whom the Software is furnished to do so, +subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS +FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR +COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER +IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN +CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/vendor/github.com/robfig/cron/README.md b/vendor/github.com/robfig/cron/README.md new file mode 100644 index 00000000000..ec40c95fcb9 --- /dev/null +++ b/vendor/github.com/robfig/cron/README.md @@ -0,0 +1,6 @@ +[![GoDoc](http://godoc.org/github.com/robfig/cron?status.png)](http://godoc.org/github.com/robfig/cron) +[![Build Status](https://travis-ci.org/robfig/cron.svg?branch=master)](https://travis-ci.org/robfig/cron) + +# cron + +Documentation here: https://godoc.org/github.com/robfig/cron diff --git a/vendor/github.com/robfig/cron/constantdelay.go b/vendor/github.com/robfig/cron/constantdelay.go new file mode 100644 index 00000000000..cd6e7b1be91 --- /dev/null +++ b/vendor/github.com/robfig/cron/constantdelay.go @@ -0,0 +1,27 @@ +package cron + +import "time" + +// ConstantDelaySchedule represents a simple recurring duty cycle, e.g. "Every 5 minutes". +// It does not support jobs more frequent than once a second. +type ConstantDelaySchedule struct { + Delay time.Duration +} + +// Every returns a crontab Schedule that activates once every duration. +// Delays of less than a second are not supported (will round up to 1 second). +// Any fields less than a Second are truncated. +func Every(duration time.Duration) ConstantDelaySchedule { + if duration < time.Second { + duration = time.Second + } + return ConstantDelaySchedule{ + Delay: duration - time.Duration(duration.Nanoseconds())%time.Second, + } +} + +// Next returns the next time this should be run. +// This rounds so that the next activation time will be on the second. +func (schedule ConstantDelaySchedule) Next(t time.Time) time.Time { + return t.Add(schedule.Delay - time.Duration(t.Nanosecond())*time.Nanosecond) +} diff --git a/vendor/github.com/robfig/cron/cron.go b/vendor/github.com/robfig/cron/cron.go new file mode 100644 index 00000000000..2318aeb2e7d --- /dev/null +++ b/vendor/github.com/robfig/cron/cron.go @@ -0,0 +1,259 @@ +package cron + +import ( + "log" + "runtime" + "sort" + "time" +) + +// Cron keeps track of any number of entries, invoking the associated func as +// specified by the schedule. It may be started, stopped, and the entries may +// be inspected while running. +type Cron struct { + entries []*Entry + stop chan struct{} + add chan *Entry + snapshot chan []*Entry + running bool + ErrorLog *log.Logger + location *time.Location +} + +// Job is an interface for submitted cron jobs. +type Job interface { + Run() +} + +// The Schedule describes a job's duty cycle. +type Schedule interface { + // Return the next activation time, later than the given time. + // Next is invoked initially, and then each time the job is run. + Next(time.Time) time.Time +} + +// Entry consists of a schedule and the func to execute on that schedule. +type Entry struct { + // The schedule on which this job should be run. + Schedule Schedule + + // The next time the job will run. This is the zero time if Cron has not been + // started or this entry's schedule is unsatisfiable + Next time.Time + + // The last time this job was run. This is the zero time if the job has never + // been run. + Prev time.Time + + // The Job to run. + Job Job +} + +// byTime is a wrapper for sorting the entry array by time +// (with zero time at the end). +type byTime []*Entry + +func (s byTime) Len() int { return len(s) } +func (s byTime) Swap(i, j int) { s[i], s[j] = s[j], s[i] } +func (s byTime) Less(i, j int) bool { + // Two zero times should return false. + // Otherwise, zero is "greater" than any other time. + // (To sort it at the end of the list.) + if s[i].Next.IsZero() { + return false + } + if s[j].Next.IsZero() { + return true + } + return s[i].Next.Before(s[j].Next) +} + +// New returns a new Cron job runner, in the Local time zone. +func New() *Cron { + return NewWithLocation(time.Now().Location()) +} + +// NewWithLocation returns a new Cron job runner. +func NewWithLocation(location *time.Location) *Cron { + return &Cron{ + entries: nil, + add: make(chan *Entry), + stop: make(chan struct{}), + snapshot: make(chan []*Entry), + running: false, + ErrorLog: nil, + location: location, + } +} + +// A wrapper that turns a func() into a cron.Job +type FuncJob func() + +func (f FuncJob) Run() { f() } + +// AddFunc adds a func to the Cron to be run on the given schedule. +func (c *Cron) AddFunc(spec string, cmd func()) error { + return c.AddJob(spec, FuncJob(cmd)) +} + +// AddJob adds a Job to the Cron to be run on the given schedule. +func (c *Cron) AddJob(spec string, cmd Job) error { + schedule, err := Parse(spec) + if err != nil { + return err + } + c.Schedule(schedule, cmd) + return nil +} + +// Schedule adds a Job to the Cron to be run on the given schedule. +func (c *Cron) Schedule(schedule Schedule, cmd Job) { + entry := &Entry{ + Schedule: schedule, + Job: cmd, + } + if !c.running { + c.entries = append(c.entries, entry) + return + } + + c.add <- entry +} + +// Entries returns a snapshot of the cron entries. +func (c *Cron) Entries() []*Entry { + if c.running { + c.snapshot <- nil + x := <-c.snapshot + return x + } + return c.entrySnapshot() +} + +// Location gets the time zone location +func (c *Cron) Location() *time.Location { + return c.location +} + +// Start the cron scheduler in its own go-routine, or no-op if already started. +func (c *Cron) Start() { + if c.running { + return + } + c.running = true + go c.run() +} + +// Run the cron scheduler, or no-op if already running. +func (c *Cron) Run() { + if c.running { + return + } + c.running = true + c.run() +} + +func (c *Cron) runWithRecovery(j Job) { + defer func() { + if r := recover(); r != nil { + const size = 64 << 10 + buf := make([]byte, size) + buf = buf[:runtime.Stack(buf, false)] + c.logf("cron: panic running job: %v\n%s", r, buf) + } + }() + j.Run() +} + +// Run the scheduler. this is private just due to the need to synchronize +// access to the 'running' state variable. +func (c *Cron) run() { + // Figure out the next activation times for each entry. + now := c.now() + for _, entry := range c.entries { + entry.Next = entry.Schedule.Next(now) + } + + for { + // Determine the next entry to run. + sort.Sort(byTime(c.entries)) + + var timer *time.Timer + if len(c.entries) == 0 || c.entries[0].Next.IsZero() { + // If there are no entries yet, just sleep - it still handles new entries + // and stop requests. + timer = time.NewTimer(100000 * time.Hour) + } else { + timer = time.NewTimer(c.entries[0].Next.Sub(now)) + } + + for { + select { + case now = <-timer.C: + now = now.In(c.location) + // Run every entry whose next time was less than now + for _, e := range c.entries { + if e.Next.After(now) || e.Next.IsZero() { + break + } + go c.runWithRecovery(e.Job) + e.Prev = e.Next + e.Next = e.Schedule.Next(now) + } + + case newEntry := <-c.add: + timer.Stop() + now = c.now() + newEntry.Next = newEntry.Schedule.Next(now) + c.entries = append(c.entries, newEntry) + + case <-c.snapshot: + c.snapshot <- c.entrySnapshot() + continue + + case <-c.stop: + timer.Stop() + return + } + + break + } + } +} + +// Logs an error to stderr or to the configured error log +func (c *Cron) logf(format string, args ...interface{}) { + if c.ErrorLog != nil { + c.ErrorLog.Printf(format, args...) + } else { + log.Printf(format, args...) + } +} + +// Stop stops the cron scheduler if it is running; otherwise it does nothing. +func (c *Cron) Stop() { + if !c.running { + return + } + c.stop <- struct{}{} + c.running = false +} + +// entrySnapshot returns a copy of the current cron entry list. +func (c *Cron) entrySnapshot() []*Entry { + entries := []*Entry{} + for _, e := range c.entries { + entries = append(entries, &Entry{ + Schedule: e.Schedule, + Next: e.Next, + Prev: e.Prev, + Job: e.Job, + }) + } + return entries +} + +// now returns current time in c location +func (c *Cron) now() time.Time { + return time.Now().In(c.location) +} diff --git a/vendor/github.com/robfig/cron/doc.go b/vendor/github.com/robfig/cron/doc.go new file mode 100644 index 00000000000..d02ec2f3b56 --- /dev/null +++ b/vendor/github.com/robfig/cron/doc.go @@ -0,0 +1,129 @@ +/* +Package cron implements a cron spec parser and job runner. + +Usage + +Callers may register Funcs to be invoked on a given schedule. Cron will run +them in their own goroutines. + + c := cron.New() + c.AddFunc("0 30 * * * *", func() { fmt.Println("Every hour on the half hour") }) + c.AddFunc("@hourly", func() { fmt.Println("Every hour") }) + c.AddFunc("@every 1h30m", func() { fmt.Println("Every hour thirty") }) + c.Start() + .. + // Funcs are invoked in their own goroutine, asynchronously. + ... + // Funcs may also be added to a running Cron + c.AddFunc("@daily", func() { fmt.Println("Every day") }) + .. + // Inspect the cron job entries' next and previous run times. + inspect(c.Entries()) + .. + c.Stop() // Stop the scheduler (does not stop any jobs already running). + +CRON Expression Format + +A cron expression represents a set of times, using 6 space-separated fields. + + Field name | Mandatory? | Allowed values | Allowed special characters + ---------- | ---------- | -------------- | -------------------------- + Seconds | Yes | 0-59 | * / , - + Minutes | Yes | 0-59 | * / , - + Hours | Yes | 0-23 | * / , - + Day of month | Yes | 1-31 | * / , - ? + Month | Yes | 1-12 or JAN-DEC | * / , - + Day of week | Yes | 0-6 or SUN-SAT | * / , - ? + +Note: Month and Day-of-week field values are case insensitive. "SUN", "Sun", +and "sun" are equally accepted. + +Special Characters + +Asterisk ( * ) + +The asterisk indicates that the cron expression will match for all values of the +field; e.g., using an asterisk in the 5th field (month) would indicate every +month. + +Slash ( / ) + +Slashes are used to describe increments of ranges. For example 3-59/15 in the +1st field (minutes) would indicate the 3rd minute of the hour and every 15 +minutes thereafter. The form "*\/..." is equivalent to the form "first-last/...", +that is, an increment over the largest possible range of the field. The form +"N/..." is accepted as meaning "N-MAX/...", that is, starting at N, use the +increment until the end of that specific range. It does not wrap around. + +Comma ( , ) + +Commas are used to separate items of a list. For example, using "MON,WED,FRI" in +the 5th field (day of week) would mean Mondays, Wednesdays and Fridays. + +Hyphen ( - ) + +Hyphens are used to define ranges. For example, 9-17 would indicate every +hour between 9am and 5pm inclusive. + +Question mark ( ? ) + +Question mark may be used instead of '*' for leaving either day-of-month or +day-of-week blank. + +Predefined schedules + +You may use one of several pre-defined schedules in place of a cron expression. + + Entry | Description | Equivalent To + ----- | ----------- | ------------- + @yearly (or @annually) | Run once a year, midnight, Jan. 1st | 0 0 0 1 1 * + @monthly | Run once a month, midnight, first of month | 0 0 0 1 * * + @weekly | Run once a week, midnight between Sat/Sun | 0 0 0 * * 0 + @daily (or @midnight) | Run once a day, midnight | 0 0 0 * * * + @hourly | Run once an hour, beginning of hour | 0 0 * * * * + +Intervals + +You may also schedule a job to execute at fixed intervals, starting at the time it's added +or cron is run. This is supported by formatting the cron spec like this: + + @every + +where "duration" is a string accepted by time.ParseDuration +(http://golang.org/pkg/time/#ParseDuration). + +For example, "@every 1h30m10s" would indicate a schedule that activates after +1 hour, 30 minutes, 10 seconds, and then every interval after that. + +Note: The interval does not take the job runtime into account. For example, +if a job takes 3 minutes to run, and it is scheduled to run every 5 minutes, +it will have only 2 minutes of idle time between each run. + +Time zones + +All interpretation and scheduling is done in the machine's local time zone (as +provided by the Go time package (http://www.golang.org/pkg/time). + +Be aware that jobs scheduled during daylight-savings leap-ahead transitions will +not be run! + +Thread safety + +Since the Cron service runs concurrently with the calling code, some amount of +care must be taken to ensure proper synchronization. + +All cron methods are designed to be correctly synchronized as long as the caller +ensures that invocations have a clear happens-before ordering between them. + +Implementation + +Cron entries are stored in an array, sorted by their next activation time. Cron +sleeps until the next job is due to be run. + +Upon waking: + - it runs each entry that is active on that second + - it calculates the next run times for the jobs that were run + - it re-sorts the array of entries by next activation time. + - it goes to sleep until the soonest job. +*/ +package cron diff --git a/vendor/github.com/robfig/cron/parser.go b/vendor/github.com/robfig/cron/parser.go new file mode 100644 index 00000000000..a5e83c0a8dc --- /dev/null +++ b/vendor/github.com/robfig/cron/parser.go @@ -0,0 +1,380 @@ +package cron + +import ( + "fmt" + "math" + "strconv" + "strings" + "time" +) + +// Configuration options for creating a parser. Most options specify which +// fields should be included, while others enable features. If a field is not +// included the parser will assume a default value. These options do not change +// the order fields are parse in. +type ParseOption int + +const ( + Second ParseOption = 1 << iota // Seconds field, default 0 + Minute // Minutes field, default 0 + Hour // Hours field, default 0 + Dom // Day of month field, default * + Month // Month field, default * + Dow // Day of week field, default * + DowOptional // Optional day of week field, default * + Descriptor // Allow descriptors such as @monthly, @weekly, etc. +) + +var places = []ParseOption{ + Second, + Minute, + Hour, + Dom, + Month, + Dow, +} + +var defaults = []string{ + "0", + "0", + "0", + "*", + "*", + "*", +} + +// A custom Parser that can be configured. +type Parser struct { + options ParseOption + optionals int +} + +// Creates a custom Parser with custom options. +// +// // Standard parser without descriptors +// specParser := NewParser(Minute | Hour | Dom | Month | Dow) +// sched, err := specParser.Parse("0 0 15 */3 *") +// +// // Same as above, just excludes time fields +// subsParser := NewParser(Dom | Month | Dow) +// sched, err := specParser.Parse("15 */3 *") +// +// // Same as above, just makes Dow optional +// subsParser := NewParser(Dom | Month | DowOptional) +// sched, err := specParser.Parse("15 */3") +// +func NewParser(options ParseOption) Parser { + optionals := 0 + if options&DowOptional > 0 { + options |= Dow + optionals++ + } + return Parser{options, optionals} +} + +// Parse returns a new crontab schedule representing the given spec. +// It returns a descriptive error if the spec is not valid. +// It accepts crontab specs and features configured by NewParser. +func (p Parser) Parse(spec string) (Schedule, error) { + if len(spec) == 0 { + return nil, fmt.Errorf("Empty spec string") + } + if spec[0] == '@' && p.options&Descriptor > 0 { + return parseDescriptor(spec) + } + + // Figure out how many fields we need + max := 0 + for _, place := range places { + if p.options&place > 0 { + max++ + } + } + min := max - p.optionals + + // Split fields on whitespace + fields := strings.Fields(spec) + + // Validate number of fields + if count := len(fields); count < min || count > max { + if min == max { + return nil, fmt.Errorf("Expected exactly %d fields, found %d: %s", min, count, spec) + } + return nil, fmt.Errorf("Expected %d to %d fields, found %d: %s", min, max, count, spec) + } + + // Fill in missing fields + fields = expandFields(fields, p.options) + + var err error + field := func(field string, r bounds) uint64 { + if err != nil { + return 0 + } + var bits uint64 + bits, err = getField(field, r) + return bits + } + + var ( + second = field(fields[0], seconds) + minute = field(fields[1], minutes) + hour = field(fields[2], hours) + dayofmonth = field(fields[3], dom) + month = field(fields[4], months) + dayofweek = field(fields[5], dow) + ) + if err != nil { + return nil, err + } + + return &SpecSchedule{ + Second: second, + Minute: minute, + Hour: hour, + Dom: dayofmonth, + Month: month, + Dow: dayofweek, + }, nil +} + +func expandFields(fields []string, options ParseOption) []string { + n := 0 + count := len(fields) + expFields := make([]string, len(places)) + copy(expFields, defaults) + for i, place := range places { + if options&place > 0 { + expFields[i] = fields[n] + n++ + } + if n == count { + break + } + } + return expFields +} + +var standardParser = NewParser( + Minute | Hour | Dom | Month | Dow | Descriptor, +) + +// ParseStandard returns a new crontab schedule representing the given standardSpec +// (https://en.wikipedia.org/wiki/Cron). It differs from Parse requiring to always +// pass 5 entries representing: minute, hour, day of month, month and day of week, +// in that order. It returns a descriptive error if the spec is not valid. +// +// It accepts +// - Standard crontab specs, e.g. "* * * * ?" +// - Descriptors, e.g. "@midnight", "@every 1h30m" +func ParseStandard(standardSpec string) (Schedule, error) { + return standardParser.Parse(standardSpec) +} + +var defaultParser = NewParser( + Second | Minute | Hour | Dom | Month | DowOptional | Descriptor, +) + +// Parse returns a new crontab schedule representing the given spec. +// It returns a descriptive error if the spec is not valid. +// +// It accepts +// - Full crontab specs, e.g. "* * * * * ?" +// - Descriptors, e.g. "@midnight", "@every 1h30m" +func Parse(spec string) (Schedule, error) { + return defaultParser.Parse(spec) +} + +// getField returns an Int with the bits set representing all of the times that +// the field represents or error parsing field value. A "field" is a comma-separated +// list of "ranges". +func getField(field string, r bounds) (uint64, error) { + var bits uint64 + ranges := strings.FieldsFunc(field, func(r rune) bool { return r == ',' }) + for _, expr := range ranges { + bit, err := getRange(expr, r) + if err != nil { + return bits, err + } + bits |= bit + } + return bits, nil +} + +// getRange returns the bits indicated by the given expression: +// number | number "-" number [ "/" number ] +// or error parsing range. +func getRange(expr string, r bounds) (uint64, error) { + var ( + start, end, step uint + rangeAndStep = strings.Split(expr, "/") + lowAndHigh = strings.Split(rangeAndStep[0], "-") + singleDigit = len(lowAndHigh) == 1 + err error + ) + + var extra uint64 + if lowAndHigh[0] == "*" || lowAndHigh[0] == "?" { + start = r.min + end = r.max + extra = starBit + } else { + start, err = parseIntOrName(lowAndHigh[0], r.names) + if err != nil { + return 0, err + } + switch len(lowAndHigh) { + case 1: + end = start + case 2: + end, err = parseIntOrName(lowAndHigh[1], r.names) + if err != nil { + return 0, err + } + default: + return 0, fmt.Errorf("Too many hyphens: %s", expr) + } + } + + switch len(rangeAndStep) { + case 1: + step = 1 + case 2: + step, err = mustParseInt(rangeAndStep[1]) + if err != nil { + return 0, err + } + + // Special handling: "N/step" means "N-max/step". + if singleDigit { + end = r.max + } + default: + return 0, fmt.Errorf("Too many slashes: %s", expr) + } + + if start < r.min { + return 0, fmt.Errorf("Beginning of range (%d) below minimum (%d): %s", start, r.min, expr) + } + if end > r.max { + return 0, fmt.Errorf("End of range (%d) above maximum (%d): %s", end, r.max, expr) + } + if start > end { + return 0, fmt.Errorf("Beginning of range (%d) beyond end of range (%d): %s", start, end, expr) + } + if step == 0 { + return 0, fmt.Errorf("Step of range should be a positive number: %s", expr) + } + + return getBits(start, end, step) | extra, nil +} + +// parseIntOrName returns the (possibly-named) integer contained in expr. +func parseIntOrName(expr string, names map[string]uint) (uint, error) { + if names != nil { + if namedInt, ok := names[strings.ToLower(expr)]; ok { + return namedInt, nil + } + } + return mustParseInt(expr) +} + +// mustParseInt parses the given expression as an int or returns an error. +func mustParseInt(expr string) (uint, error) { + num, err := strconv.Atoi(expr) + if err != nil { + return 0, fmt.Errorf("Failed to parse int from %s: %s", expr, err) + } + if num < 0 { + return 0, fmt.Errorf("Negative number (%d) not allowed: %s", num, expr) + } + + return uint(num), nil +} + +// getBits sets all bits in the range [min, max], modulo the given step size. +func getBits(min, max, step uint) uint64 { + var bits uint64 + + // If step is 1, use shifts. + if step == 1 { + return ^(math.MaxUint64 << (max + 1)) & (math.MaxUint64 << min) + } + + // Else, use a simple loop. + for i := min; i <= max; i += step { + bits |= 1 << i + } + return bits +} + +// all returns all bits within the given bounds. (plus the star bit) +func all(r bounds) uint64 { + return getBits(r.min, r.max, 1) | starBit +} + +// parseDescriptor returns a predefined schedule for the expression, or error if none matches. +func parseDescriptor(descriptor string) (Schedule, error) { + switch descriptor { + case "@yearly", "@annually": + return &SpecSchedule{ + Second: 1 << seconds.min, + Minute: 1 << minutes.min, + Hour: 1 << hours.min, + Dom: 1 << dom.min, + Month: 1 << months.min, + Dow: all(dow), + }, nil + + case "@monthly": + return &SpecSchedule{ + Second: 1 << seconds.min, + Minute: 1 << minutes.min, + Hour: 1 << hours.min, + Dom: 1 << dom.min, + Month: all(months), + Dow: all(dow), + }, nil + + case "@weekly": + return &SpecSchedule{ + Second: 1 << seconds.min, + Minute: 1 << minutes.min, + Hour: 1 << hours.min, + Dom: all(dom), + Month: all(months), + Dow: 1 << dow.min, + }, nil + + case "@daily", "@midnight": + return &SpecSchedule{ + Second: 1 << seconds.min, + Minute: 1 << minutes.min, + Hour: 1 << hours.min, + Dom: all(dom), + Month: all(months), + Dow: all(dow), + }, nil + + case "@hourly": + return &SpecSchedule{ + Second: 1 << seconds.min, + Minute: 1 << minutes.min, + Hour: all(hours), + Dom: all(dom), + Month: all(months), + Dow: all(dow), + }, nil + } + + const every = "@every " + if strings.HasPrefix(descriptor, every) { + duration, err := time.ParseDuration(descriptor[len(every):]) + if err != nil { + return nil, fmt.Errorf("Failed to parse duration %s: %s", descriptor, err) + } + return Every(duration), nil + } + + return nil, fmt.Errorf("Unrecognized descriptor: %s", descriptor) +} diff --git a/vendor/github.com/robfig/cron/spec.go b/vendor/github.com/robfig/cron/spec.go new file mode 100644 index 00000000000..aac9a60b954 --- /dev/null +++ b/vendor/github.com/robfig/cron/spec.go @@ -0,0 +1,158 @@ +package cron + +import "time" + +// SpecSchedule specifies a duty cycle (to the second granularity), based on a +// traditional crontab specification. It is computed initially and stored as bit sets. +type SpecSchedule struct { + Second, Minute, Hour, Dom, Month, Dow uint64 +} + +// bounds provides a range of acceptable values (plus a map of name to value). +type bounds struct { + min, max uint + names map[string]uint +} + +// The bounds for each field. +var ( + seconds = bounds{0, 59, nil} + minutes = bounds{0, 59, nil} + hours = bounds{0, 23, nil} + dom = bounds{1, 31, nil} + months = bounds{1, 12, map[string]uint{ + "jan": 1, + "feb": 2, + "mar": 3, + "apr": 4, + "may": 5, + "jun": 6, + "jul": 7, + "aug": 8, + "sep": 9, + "oct": 10, + "nov": 11, + "dec": 12, + }} + dow = bounds{0, 6, map[string]uint{ + "sun": 0, + "mon": 1, + "tue": 2, + "wed": 3, + "thu": 4, + "fri": 5, + "sat": 6, + }} +) + +const ( + // Set the top bit if a star was included in the expression. + starBit = 1 << 63 +) + +// Next returns the next time this schedule is activated, greater than the given +// time. If no time can be found to satisfy the schedule, return the zero time. +func (s *SpecSchedule) Next(t time.Time) time.Time { + // General approach: + // For Month, Day, Hour, Minute, Second: + // Check if the time value matches. If yes, continue to the next field. + // If the field doesn't match the schedule, then increment the field until it matches. + // While incrementing the field, a wrap-around brings it back to the beginning + // of the field list (since it is necessary to re-verify previous field + // values) + + // Start at the earliest possible time (the upcoming second). + t = t.Add(1*time.Second - time.Duration(t.Nanosecond())*time.Nanosecond) + + // This flag indicates whether a field has been incremented. + added := false + + // If no time is found within five years, return zero. + yearLimit := t.Year() + 5 + +WRAP: + if t.Year() > yearLimit { + return time.Time{} + } + + // Find the first applicable month. + // If it's this month, then do nothing. + for 1< 0 + dowMatch bool = 1< 0 + ) + if s.Dom&starBit > 0 || s.Dow&starBit > 0 { + return domMatch && dowMatch + } + return domMatch || dowMatch +} diff --git a/vendor/go.temporal.io/api/LICENSE b/vendor/go.temporal.io/api/LICENSE new file mode 100644 index 00000000000..c7f1b9e7e80 --- /dev/null +++ b/vendor/go.temporal.io/api/LICENSE @@ -0,0 +1,21 @@ +The MIT License + +Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. \ No newline at end of file diff --git a/vendor/go.temporal.io/api/batch/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/batch/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..3d7ca69e4d4 --- /dev/null +++ b/vendor/go.temporal.io/api/batch/v1/message.go-helpers.pb.go @@ -0,0 +1,250 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package batch + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type BatchOperationInfo to the protobuf v3 wire format +func (val *BatchOperationInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type BatchOperationInfo from the protobuf v3 wire format +func (val *BatchOperationInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *BatchOperationInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two BatchOperationInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *BatchOperationInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *BatchOperationInfo + switch t := that.(type) { + case *BatchOperationInfo: + that1 = t + case BatchOperationInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type BatchOperationTermination to the protobuf v3 wire format +func (val *BatchOperationTermination) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type BatchOperationTermination from the protobuf v3 wire format +func (val *BatchOperationTermination) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *BatchOperationTermination) Size() int { + return proto.Size(val) +} + +// Equal returns whether two BatchOperationTermination values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *BatchOperationTermination) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *BatchOperationTermination + switch t := that.(type) { + case *BatchOperationTermination: + that1 = t + case BatchOperationTermination: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type BatchOperationSignal to the protobuf v3 wire format +func (val *BatchOperationSignal) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type BatchOperationSignal from the protobuf v3 wire format +func (val *BatchOperationSignal) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *BatchOperationSignal) Size() int { + return proto.Size(val) +} + +// Equal returns whether two BatchOperationSignal values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *BatchOperationSignal) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *BatchOperationSignal + switch t := that.(type) { + case *BatchOperationSignal: + that1 = t + case BatchOperationSignal: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type BatchOperationCancellation to the protobuf v3 wire format +func (val *BatchOperationCancellation) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type BatchOperationCancellation from the protobuf v3 wire format +func (val *BatchOperationCancellation) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *BatchOperationCancellation) Size() int { + return proto.Size(val) +} + +// Equal returns whether two BatchOperationCancellation values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *BatchOperationCancellation) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *BatchOperationCancellation + switch t := that.(type) { + case *BatchOperationCancellation: + that1 = t + case BatchOperationCancellation: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type BatchOperationDeletion to the protobuf v3 wire format +func (val *BatchOperationDeletion) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type BatchOperationDeletion from the protobuf v3 wire format +func (val *BatchOperationDeletion) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *BatchOperationDeletion) Size() int { + return proto.Size(val) +} + +// Equal returns whether two BatchOperationDeletion values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *BatchOperationDeletion) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *BatchOperationDeletion + switch t := that.(type) { + case *BatchOperationDeletion: + that1 = t + case BatchOperationDeletion: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type BatchOperationReset to the protobuf v3 wire format +func (val *BatchOperationReset) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type BatchOperationReset from the protobuf v3 wire format +func (val *BatchOperationReset) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *BatchOperationReset) Size() int { + return proto.Size(val) +} + +// Equal returns whether two BatchOperationReset values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *BatchOperationReset) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *BatchOperationReset + switch t := that.(type) { + case *BatchOperationReset: + that1 = t + case BatchOperationReset: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/batch/v1/message.pb.go b/vendor/go.temporal.io/api/batch/v1/message.pb.go new file mode 100644 index 00000000000..07cb0ddcd11 --- /dev/null +++ b/vendor/go.temporal.io/api/batch/v1/message.pb.go @@ -0,0 +1,667 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/batch/v1/message.proto + +package batch + +import ( + reflect "reflect" + sync "sync" + + v11 "go.temporal.io/api/common/v1" + v1 "go.temporal.io/api/enums/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type BatchOperationInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Batch job ID + JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` + // Batch operation state + State v1.BatchOperationState `protobuf:"varint,2,opt,name=state,proto3,enum=temporal.api.enums.v1.BatchOperationState" json:"state,omitempty"` + // Batch operation start time + StartTime *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + // Batch operation close time + CloseTime *timestamppb.Timestamp `protobuf:"bytes,4,opt,name=close_time,json=closeTime,proto3" json:"close_time,omitempty"` +} + +func (x *BatchOperationInfo) Reset() { + *x = BatchOperationInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_batch_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BatchOperationInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BatchOperationInfo) ProtoMessage() {} + +func (x *BatchOperationInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_batch_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BatchOperationInfo.ProtoReflect.Descriptor instead. +func (*BatchOperationInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_batch_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *BatchOperationInfo) GetJobId() string { + if x != nil { + return x.JobId + } + return "" +} + +func (x *BatchOperationInfo) GetState() v1.BatchOperationState { + if x != nil { + return x.State + } + return v1.BatchOperationState(0) +} + +func (x *BatchOperationInfo) GetStartTime() *timestamppb.Timestamp { + if x != nil { + return x.StartTime + } + return nil +} + +func (x *BatchOperationInfo) GetCloseTime() *timestamppb.Timestamp { + if x != nil { + return x.CloseTime + } + return nil +} + +// BatchOperationTermination sends terminate requests to batch workflows. +// Keep the parameter in sync with temporal.api.workflowservice.v1.TerminateWorkflowExecutionRequest. +// Ignore first_execution_run_id because this is used for single workflow operation. +type BatchOperationTermination struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Serialized value(s) to provide to the termination event + Details *v11.Payloads `protobuf:"bytes,1,opt,name=details,proto3" json:"details,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,2,opt,name=identity,proto3" json:"identity,omitempty"` +} + +func (x *BatchOperationTermination) Reset() { + *x = BatchOperationTermination{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_batch_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BatchOperationTermination) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BatchOperationTermination) ProtoMessage() {} + +func (x *BatchOperationTermination) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_batch_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BatchOperationTermination.ProtoReflect.Descriptor instead. +func (*BatchOperationTermination) Descriptor() ([]byte, []int) { + return file_temporal_api_batch_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *BatchOperationTermination) GetDetails() *v11.Payloads { + if x != nil { + return x.Details + } + return nil +} + +func (x *BatchOperationTermination) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +// BatchOperationSignal sends signals to batch workflows. +// Keep the parameter in sync with temporal.api.workflowservice.v1.SignalWorkflowExecutionRequest. +type BatchOperationSignal struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The workflow author-defined name of the signal to send to the workflow + Signal string `protobuf:"bytes,1,opt,name=signal,proto3" json:"signal,omitempty"` + // Serialized value(s) to provide with the signal + Input *v11.Payloads `protobuf:"bytes,2,opt,name=input,proto3" json:"input,omitempty"` + // Headers that are passed with the signal to the processing workflow. + // These can include things like auth or tracing tokens. + Header *v11.Header `protobuf:"bytes,3,opt,name=header,proto3" json:"header,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,4,opt,name=identity,proto3" json:"identity,omitempty"` +} + +func (x *BatchOperationSignal) Reset() { + *x = BatchOperationSignal{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_batch_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BatchOperationSignal) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BatchOperationSignal) ProtoMessage() {} + +func (x *BatchOperationSignal) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_batch_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BatchOperationSignal.ProtoReflect.Descriptor instead. +func (*BatchOperationSignal) Descriptor() ([]byte, []int) { + return file_temporal_api_batch_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *BatchOperationSignal) GetSignal() string { + if x != nil { + return x.Signal + } + return "" +} + +func (x *BatchOperationSignal) GetInput() *v11.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *BatchOperationSignal) GetHeader() *v11.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *BatchOperationSignal) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +// BatchOperationCancellation sends cancel requests to batch workflows. +// Keep the parameter in sync with temporal.api.workflowservice.v1.RequestCancelWorkflowExecutionRequest. +// Ignore first_execution_run_id because this is used for single workflow operation. +type BatchOperationCancellation struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The identity of the worker/client + Identity string `protobuf:"bytes,1,opt,name=identity,proto3" json:"identity,omitempty"` +} + +func (x *BatchOperationCancellation) Reset() { + *x = BatchOperationCancellation{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_batch_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BatchOperationCancellation) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BatchOperationCancellation) ProtoMessage() {} + +func (x *BatchOperationCancellation) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_batch_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BatchOperationCancellation.ProtoReflect.Descriptor instead. +func (*BatchOperationCancellation) Descriptor() ([]byte, []int) { + return file_temporal_api_batch_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *BatchOperationCancellation) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +// BatchOperationDeletion sends deletion requests to batch workflows. +// Keep the parameter in sync with temporal.api.workflowservice.v1.DeleteWorkflowExecutionRequest. +type BatchOperationDeletion struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The identity of the worker/client + Identity string `protobuf:"bytes,1,opt,name=identity,proto3" json:"identity,omitempty"` +} + +func (x *BatchOperationDeletion) Reset() { + *x = BatchOperationDeletion{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_batch_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BatchOperationDeletion) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BatchOperationDeletion) ProtoMessage() {} + +func (x *BatchOperationDeletion) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_batch_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BatchOperationDeletion.ProtoReflect.Descriptor instead. +func (*BatchOperationDeletion) Descriptor() ([]byte, []int) { + return file_temporal_api_batch_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *BatchOperationDeletion) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +// BatchOperationReset sends reset requests to batch workflows. +// Keep the parameter in sync with temporal.api.workflowservice.v1.ResetWorkflowExecutionRequest. +type BatchOperationReset struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The identity of the worker/client. + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` + // Describes what to reset to and how. If set, `reset_type` and `reset_reapply_type` are ignored. + Options *v11.ResetOptions `protobuf:"bytes,4,opt,name=options,proto3" json:"options,omitempty"` + // Reset type (deprecated, use `options`). + ResetType v1.ResetType `protobuf:"varint,1,opt,name=reset_type,json=resetType,proto3,enum=temporal.api.enums.v1.ResetType" json:"reset_type,omitempty"` + // History event reapply options (deprecated, use `options`). + ResetReapplyType v1.ResetReapplyType `protobuf:"varint,2,opt,name=reset_reapply_type,json=resetReapplyType,proto3,enum=temporal.api.enums.v1.ResetReapplyType" json:"reset_reapply_type,omitempty"` +} + +func (x *BatchOperationReset) Reset() { + *x = BatchOperationReset{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_batch_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BatchOperationReset) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BatchOperationReset) ProtoMessage() {} + +func (x *BatchOperationReset) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_batch_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BatchOperationReset.ProtoReflect.Descriptor instead. +func (*BatchOperationReset) Descriptor() ([]byte, []int) { + return file_temporal_api_batch_v1_message_proto_rawDescGZIP(), []int{5} +} + +func (x *BatchOperationReset) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *BatchOperationReset) GetOptions() *v11.ResetOptions { + if x != nil { + return x.Options + } + return nil +} + +func (x *BatchOperationReset) GetResetType() v1.ResetType { + if x != nil { + return x.ResetType + } + return v1.ResetType(0) +} + +func (x *BatchOperationReset) GetResetReapplyType() v1.ResetReapplyType { + if x != nil { + return x.ResetReapplyType + } + return v1.ResetReapplyType(0) +} + +var File_temporal_api_batch_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_batch_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x23, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x62, 0x61, + 0x74, 0x63, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x2b, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, + 0x76, 0x31, 0x2f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x21, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x72, 0x65, + 0x73, 0x65, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xf3, 0x01, 0x0a, 0x12, 0x42, 0x61, 0x74, + 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x19, + 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, + 0x6f, 0x62, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x44, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x61, + 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, + 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x0a, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x7b, 0x0a, 0x19, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3e, + 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, + 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0xca, 0x01, 0x0a, 0x14, 0x42, 0x61, + 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x69, 0x67, 0x6e, 0x61, + 0x6c, 0x12, 0x1a, 0x0a, 0x06, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x06, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, + 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, + 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, + 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x3c, 0x0a, + 0x1a, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, + 0x6e, 0x63, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x38, 0x0a, 0x16, 0x42, 0x61, 0x74, 0x63, 0x68, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, + 0x99, 0x02, 0x0a, 0x13, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, + 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, + 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x42, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, + 0x65, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x43, 0x0a, 0x0a, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, + 0x52, 0x65, 0x73, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x72, 0x65, 0x73, 0x65, 0x74, 0x54, + 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, 0x12, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, + 0x72, 0x65, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x27, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, + 0x61, 0x70, 0x70, 0x6c, 0x79, 0x54, 0x79, 0x70, 0x65, 0x52, 0x10, 0x72, 0x65, 0x73, 0x65, 0x74, 0x52, + 0x65, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x42, 0x84, 0x01, + 0x0a, 0x18, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x21, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x62, 0x61, 0x74, 0x63, + 0x68, 0x2f, 0x76, 0x31, 0x3b, 0x62, 0x61, 0x74, 0x63, 0x68, 0xaa, 0x02, 0x17, 0x54, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x2e, + 0x56, 0x31, 0xea, 0x02, 0x1a, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, + 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x42, 0x61, 0x74, 0x63, 0x68, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_batch_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_batch_v1_message_proto_rawDescData = file_temporal_api_batch_v1_message_proto_rawDesc +) + +func file_temporal_api_batch_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_batch_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_batch_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_batch_v1_message_proto_rawDescData) + }) + return file_temporal_api_batch_v1_message_proto_rawDescData +} + +var file_temporal_api_batch_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 6) +var file_temporal_api_batch_v1_message_proto_goTypes = []any{ + (*BatchOperationInfo)(nil), // 0: temporal.api.batch.v1.BatchOperationInfo + (*BatchOperationTermination)(nil), // 1: temporal.api.batch.v1.BatchOperationTermination + (*BatchOperationSignal)(nil), // 2: temporal.api.batch.v1.BatchOperationSignal + (*BatchOperationCancellation)(nil), // 3: temporal.api.batch.v1.BatchOperationCancellation + (*BatchOperationDeletion)(nil), // 4: temporal.api.batch.v1.BatchOperationDeletion + (*BatchOperationReset)(nil), // 5: temporal.api.batch.v1.BatchOperationReset + (v1.BatchOperationState)(0), // 6: temporal.api.enums.v1.BatchOperationState + (*timestamppb.Timestamp)(nil), // 7: google.protobuf.Timestamp + (*v11.Payloads)(nil), // 8: temporal.api.common.v1.Payloads + (*v11.Header)(nil), // 9: temporal.api.common.v1.Header + (*v11.ResetOptions)(nil), // 10: temporal.api.common.v1.ResetOptions + (v1.ResetType)(0), // 11: temporal.api.enums.v1.ResetType + (v1.ResetReapplyType)(0), // 12: temporal.api.enums.v1.ResetReapplyType +} +var file_temporal_api_batch_v1_message_proto_depIdxs = []int32{ + 6, // 0: temporal.api.batch.v1.BatchOperationInfo.state:type_name -> temporal.api.enums.v1.BatchOperationState + 7, // 1: temporal.api.batch.v1.BatchOperationInfo.start_time:type_name -> google.protobuf.Timestamp + 7, // 2: temporal.api.batch.v1.BatchOperationInfo.close_time:type_name -> google.protobuf.Timestamp + 8, // 3: temporal.api.batch.v1.BatchOperationTermination.details:type_name -> temporal.api.common.v1.Payloads + 8, // 4: temporal.api.batch.v1.BatchOperationSignal.input:type_name -> temporal.api.common.v1.Payloads + 9, // 5: temporal.api.batch.v1.BatchOperationSignal.header:type_name -> temporal.api.common.v1.Header + 10, // 6: temporal.api.batch.v1.BatchOperationReset.options:type_name -> temporal.api.common.v1.ResetOptions + 11, // 7: temporal.api.batch.v1.BatchOperationReset.reset_type:type_name -> temporal.api.enums.v1.ResetType + 12, // 8: temporal.api.batch.v1.BatchOperationReset.reset_reapply_type:type_name -> temporal.api.enums.v1.ResetReapplyType + 9, // [9:9] is the sub-list for method output_type + 9, // [9:9] is the sub-list for method input_type + 9, // [9:9] is the sub-list for extension type_name + 9, // [9:9] is the sub-list for extension extendee + 0, // [0:9] is the sub-list for field type_name +} + +func init() { file_temporal_api_batch_v1_message_proto_init() } +func file_temporal_api_batch_v1_message_proto_init() { + if File_temporal_api_batch_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_batch_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*BatchOperationInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_batch_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*BatchOperationTermination); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_batch_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*BatchOperationSignal); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_batch_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*BatchOperationCancellation); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_batch_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*BatchOperationDeletion); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_batch_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*BatchOperationReset); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_batch_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 6, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_batch_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_batch_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_batch_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_batch_v1_message_proto = out.File + file_temporal_api_batch_v1_message_proto_rawDesc = nil + file_temporal_api_batch_v1_message_proto_goTypes = nil + file_temporal_api_batch_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/cloud/cloudservice/v1/request_response.go-helpers.pb.go b/vendor/go.temporal.io/api/cloud/cloudservice/v1/request_response.go-helpers.pb.go new file mode 100644 index 00000000000..db14891fa47 --- /dev/null +++ b/vendor/go.temporal.io/api/cloud/cloudservice/v1/request_response.go-helpers.pb.go @@ -0,0 +1,2470 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package cloudservice + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type GetUsersRequest to the protobuf v3 wire format +func (val *GetUsersRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetUsersRequest from the protobuf v3 wire format +func (val *GetUsersRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetUsersRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetUsersRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetUsersRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetUsersRequest + switch t := that.(type) { + case *GetUsersRequest: + that1 = t + case GetUsersRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetUsersResponse to the protobuf v3 wire format +func (val *GetUsersResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetUsersResponse from the protobuf v3 wire format +func (val *GetUsersResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetUsersResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetUsersResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetUsersResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetUsersResponse + switch t := that.(type) { + case *GetUsersResponse: + that1 = t + case GetUsersResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetUserRequest to the protobuf v3 wire format +func (val *GetUserRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetUserRequest from the protobuf v3 wire format +func (val *GetUserRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetUserRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetUserRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetUserRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetUserRequest + switch t := that.(type) { + case *GetUserRequest: + that1 = t + case GetUserRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetUserResponse to the protobuf v3 wire format +func (val *GetUserResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetUserResponse from the protobuf v3 wire format +func (val *GetUserResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetUserResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetUserResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetUserResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetUserResponse + switch t := that.(type) { + case *GetUserResponse: + that1 = t + case GetUserResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateUserRequest to the protobuf v3 wire format +func (val *CreateUserRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateUserRequest from the protobuf v3 wire format +func (val *CreateUserRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateUserRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateUserRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateUserRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateUserRequest + switch t := that.(type) { + case *CreateUserRequest: + that1 = t + case CreateUserRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateUserResponse to the protobuf v3 wire format +func (val *CreateUserResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateUserResponse from the protobuf v3 wire format +func (val *CreateUserResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateUserResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateUserResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateUserResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateUserResponse + switch t := that.(type) { + case *CreateUserResponse: + that1 = t + case CreateUserResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateUserRequest to the protobuf v3 wire format +func (val *UpdateUserRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateUserRequest from the protobuf v3 wire format +func (val *UpdateUserRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateUserRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateUserRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateUserRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateUserRequest + switch t := that.(type) { + case *UpdateUserRequest: + that1 = t + case UpdateUserRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateUserResponse to the protobuf v3 wire format +func (val *UpdateUserResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateUserResponse from the protobuf v3 wire format +func (val *UpdateUserResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateUserResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateUserResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateUserResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateUserResponse + switch t := that.(type) { + case *UpdateUserResponse: + that1 = t + case UpdateUserResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteUserRequest to the protobuf v3 wire format +func (val *DeleteUserRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteUserRequest from the protobuf v3 wire format +func (val *DeleteUserRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteUserRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteUserRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteUserRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteUserRequest + switch t := that.(type) { + case *DeleteUserRequest: + that1 = t + case DeleteUserRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteUserResponse to the protobuf v3 wire format +func (val *DeleteUserResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteUserResponse from the protobuf v3 wire format +func (val *DeleteUserResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteUserResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteUserResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteUserResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteUserResponse + switch t := that.(type) { + case *DeleteUserResponse: + that1 = t + case DeleteUserResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SetUserNamespaceAccessRequest to the protobuf v3 wire format +func (val *SetUserNamespaceAccessRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SetUserNamespaceAccessRequest from the protobuf v3 wire format +func (val *SetUserNamespaceAccessRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SetUserNamespaceAccessRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SetUserNamespaceAccessRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SetUserNamespaceAccessRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SetUserNamespaceAccessRequest + switch t := that.(type) { + case *SetUserNamespaceAccessRequest: + that1 = t + case SetUserNamespaceAccessRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SetUserNamespaceAccessResponse to the protobuf v3 wire format +func (val *SetUserNamespaceAccessResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SetUserNamespaceAccessResponse from the protobuf v3 wire format +func (val *SetUserNamespaceAccessResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SetUserNamespaceAccessResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SetUserNamespaceAccessResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SetUserNamespaceAccessResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SetUserNamespaceAccessResponse + switch t := that.(type) { + case *SetUserNamespaceAccessResponse: + that1 = t + case SetUserNamespaceAccessResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetAsyncOperationRequest to the protobuf v3 wire format +func (val *GetAsyncOperationRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetAsyncOperationRequest from the protobuf v3 wire format +func (val *GetAsyncOperationRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetAsyncOperationRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetAsyncOperationRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetAsyncOperationRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetAsyncOperationRequest + switch t := that.(type) { + case *GetAsyncOperationRequest: + that1 = t + case GetAsyncOperationRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetAsyncOperationResponse to the protobuf v3 wire format +func (val *GetAsyncOperationResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetAsyncOperationResponse from the protobuf v3 wire format +func (val *GetAsyncOperationResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetAsyncOperationResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetAsyncOperationResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetAsyncOperationResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetAsyncOperationResponse + switch t := that.(type) { + case *GetAsyncOperationResponse: + that1 = t + case GetAsyncOperationResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateNamespaceRequest to the protobuf v3 wire format +func (val *CreateNamespaceRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateNamespaceRequest from the protobuf v3 wire format +func (val *CreateNamespaceRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateNamespaceRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateNamespaceRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateNamespaceRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateNamespaceRequest + switch t := that.(type) { + case *CreateNamespaceRequest: + that1 = t + case CreateNamespaceRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateNamespaceResponse to the protobuf v3 wire format +func (val *CreateNamespaceResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateNamespaceResponse from the protobuf v3 wire format +func (val *CreateNamespaceResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateNamespaceResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateNamespaceResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateNamespaceResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateNamespaceResponse + switch t := that.(type) { + case *CreateNamespaceResponse: + that1 = t + case CreateNamespaceResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetNamespacesRequest to the protobuf v3 wire format +func (val *GetNamespacesRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetNamespacesRequest from the protobuf v3 wire format +func (val *GetNamespacesRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetNamespacesRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetNamespacesRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetNamespacesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetNamespacesRequest + switch t := that.(type) { + case *GetNamespacesRequest: + that1 = t + case GetNamespacesRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetNamespacesResponse to the protobuf v3 wire format +func (val *GetNamespacesResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetNamespacesResponse from the protobuf v3 wire format +func (val *GetNamespacesResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetNamespacesResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetNamespacesResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetNamespacesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetNamespacesResponse + switch t := that.(type) { + case *GetNamespacesResponse: + that1 = t + case GetNamespacesResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetNamespaceRequest to the protobuf v3 wire format +func (val *GetNamespaceRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetNamespaceRequest from the protobuf v3 wire format +func (val *GetNamespaceRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetNamespaceRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetNamespaceRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetNamespaceRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetNamespaceRequest + switch t := that.(type) { + case *GetNamespaceRequest: + that1 = t + case GetNamespaceRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetNamespaceResponse to the protobuf v3 wire format +func (val *GetNamespaceResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetNamespaceResponse from the protobuf v3 wire format +func (val *GetNamespaceResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetNamespaceResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetNamespaceResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetNamespaceResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetNamespaceResponse + switch t := that.(type) { + case *GetNamespaceResponse: + that1 = t + case GetNamespaceResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateNamespaceRequest to the protobuf v3 wire format +func (val *UpdateNamespaceRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateNamespaceRequest from the protobuf v3 wire format +func (val *UpdateNamespaceRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateNamespaceRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateNamespaceRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateNamespaceRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateNamespaceRequest + switch t := that.(type) { + case *UpdateNamespaceRequest: + that1 = t + case UpdateNamespaceRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateNamespaceResponse to the protobuf v3 wire format +func (val *UpdateNamespaceResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateNamespaceResponse from the protobuf v3 wire format +func (val *UpdateNamespaceResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateNamespaceResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateNamespaceResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateNamespaceResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateNamespaceResponse + switch t := that.(type) { + case *UpdateNamespaceResponse: + that1 = t + case UpdateNamespaceResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RenameCustomSearchAttributeRequest to the protobuf v3 wire format +func (val *RenameCustomSearchAttributeRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RenameCustomSearchAttributeRequest from the protobuf v3 wire format +func (val *RenameCustomSearchAttributeRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RenameCustomSearchAttributeRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RenameCustomSearchAttributeRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RenameCustomSearchAttributeRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RenameCustomSearchAttributeRequest + switch t := that.(type) { + case *RenameCustomSearchAttributeRequest: + that1 = t + case RenameCustomSearchAttributeRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RenameCustomSearchAttributeResponse to the protobuf v3 wire format +func (val *RenameCustomSearchAttributeResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RenameCustomSearchAttributeResponse from the protobuf v3 wire format +func (val *RenameCustomSearchAttributeResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RenameCustomSearchAttributeResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RenameCustomSearchAttributeResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RenameCustomSearchAttributeResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RenameCustomSearchAttributeResponse + switch t := that.(type) { + case *RenameCustomSearchAttributeResponse: + that1 = t + case RenameCustomSearchAttributeResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteNamespaceRequest to the protobuf v3 wire format +func (val *DeleteNamespaceRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteNamespaceRequest from the protobuf v3 wire format +func (val *DeleteNamespaceRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteNamespaceRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteNamespaceRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteNamespaceRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteNamespaceRequest + switch t := that.(type) { + case *DeleteNamespaceRequest: + that1 = t + case DeleteNamespaceRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteNamespaceResponse to the protobuf v3 wire format +func (val *DeleteNamespaceResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteNamespaceResponse from the protobuf v3 wire format +func (val *DeleteNamespaceResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteNamespaceResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteNamespaceResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteNamespaceResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteNamespaceResponse + switch t := that.(type) { + case *DeleteNamespaceResponse: + that1 = t + case DeleteNamespaceResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type FailoverNamespaceRegionRequest to the protobuf v3 wire format +func (val *FailoverNamespaceRegionRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type FailoverNamespaceRegionRequest from the protobuf v3 wire format +func (val *FailoverNamespaceRegionRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *FailoverNamespaceRegionRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two FailoverNamespaceRegionRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *FailoverNamespaceRegionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *FailoverNamespaceRegionRequest + switch t := that.(type) { + case *FailoverNamespaceRegionRequest: + that1 = t + case FailoverNamespaceRegionRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type FailoverNamespaceRegionResponse to the protobuf v3 wire format +func (val *FailoverNamespaceRegionResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type FailoverNamespaceRegionResponse from the protobuf v3 wire format +func (val *FailoverNamespaceRegionResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *FailoverNamespaceRegionResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two FailoverNamespaceRegionResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *FailoverNamespaceRegionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *FailoverNamespaceRegionResponse + switch t := that.(type) { + case *FailoverNamespaceRegionResponse: + that1 = t + case FailoverNamespaceRegionResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type AddNamespaceRegionRequest to the protobuf v3 wire format +func (val *AddNamespaceRegionRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type AddNamespaceRegionRequest from the protobuf v3 wire format +func (val *AddNamespaceRegionRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *AddNamespaceRegionRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two AddNamespaceRegionRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *AddNamespaceRegionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *AddNamespaceRegionRequest + switch t := that.(type) { + case *AddNamespaceRegionRequest: + that1 = t + case AddNamespaceRegionRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type AddNamespaceRegionResponse to the protobuf v3 wire format +func (val *AddNamespaceRegionResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type AddNamespaceRegionResponse from the protobuf v3 wire format +func (val *AddNamespaceRegionResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *AddNamespaceRegionResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two AddNamespaceRegionResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *AddNamespaceRegionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *AddNamespaceRegionResponse + switch t := that.(type) { + case *AddNamespaceRegionResponse: + that1 = t + case AddNamespaceRegionResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetRegionsRequest to the protobuf v3 wire format +func (val *GetRegionsRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetRegionsRequest from the protobuf v3 wire format +func (val *GetRegionsRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetRegionsRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetRegionsRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetRegionsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetRegionsRequest + switch t := that.(type) { + case *GetRegionsRequest: + that1 = t + case GetRegionsRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetRegionsResponse to the protobuf v3 wire format +func (val *GetRegionsResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetRegionsResponse from the protobuf v3 wire format +func (val *GetRegionsResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetRegionsResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetRegionsResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetRegionsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetRegionsResponse + switch t := that.(type) { + case *GetRegionsResponse: + that1 = t + case GetRegionsResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetRegionRequest to the protobuf v3 wire format +func (val *GetRegionRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetRegionRequest from the protobuf v3 wire format +func (val *GetRegionRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetRegionRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetRegionRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetRegionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetRegionRequest + switch t := that.(type) { + case *GetRegionRequest: + that1 = t + case GetRegionRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetRegionResponse to the protobuf v3 wire format +func (val *GetRegionResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetRegionResponse from the protobuf v3 wire format +func (val *GetRegionResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetRegionResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetRegionResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetRegionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetRegionResponse + switch t := that.(type) { + case *GetRegionResponse: + that1 = t + case GetRegionResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetApiKeysRequest to the protobuf v3 wire format +func (val *GetApiKeysRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetApiKeysRequest from the protobuf v3 wire format +func (val *GetApiKeysRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetApiKeysRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetApiKeysRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetApiKeysRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetApiKeysRequest + switch t := that.(type) { + case *GetApiKeysRequest: + that1 = t + case GetApiKeysRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetApiKeysResponse to the protobuf v3 wire format +func (val *GetApiKeysResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetApiKeysResponse from the protobuf v3 wire format +func (val *GetApiKeysResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetApiKeysResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetApiKeysResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetApiKeysResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetApiKeysResponse + switch t := that.(type) { + case *GetApiKeysResponse: + that1 = t + case GetApiKeysResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetApiKeyRequest to the protobuf v3 wire format +func (val *GetApiKeyRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetApiKeyRequest from the protobuf v3 wire format +func (val *GetApiKeyRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetApiKeyRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetApiKeyRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetApiKeyRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetApiKeyRequest + switch t := that.(type) { + case *GetApiKeyRequest: + that1 = t + case GetApiKeyRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetApiKeyResponse to the protobuf v3 wire format +func (val *GetApiKeyResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetApiKeyResponse from the protobuf v3 wire format +func (val *GetApiKeyResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetApiKeyResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetApiKeyResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetApiKeyResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetApiKeyResponse + switch t := that.(type) { + case *GetApiKeyResponse: + that1 = t + case GetApiKeyResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateApiKeyRequest to the protobuf v3 wire format +func (val *CreateApiKeyRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateApiKeyRequest from the protobuf v3 wire format +func (val *CreateApiKeyRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateApiKeyRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateApiKeyRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateApiKeyRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateApiKeyRequest + switch t := that.(type) { + case *CreateApiKeyRequest: + that1 = t + case CreateApiKeyRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateApiKeyResponse to the protobuf v3 wire format +func (val *CreateApiKeyResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateApiKeyResponse from the protobuf v3 wire format +func (val *CreateApiKeyResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateApiKeyResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateApiKeyResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateApiKeyResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateApiKeyResponse + switch t := that.(type) { + case *CreateApiKeyResponse: + that1 = t + case CreateApiKeyResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateApiKeyRequest to the protobuf v3 wire format +func (val *UpdateApiKeyRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateApiKeyRequest from the protobuf v3 wire format +func (val *UpdateApiKeyRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateApiKeyRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateApiKeyRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateApiKeyRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateApiKeyRequest + switch t := that.(type) { + case *UpdateApiKeyRequest: + that1 = t + case UpdateApiKeyRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateApiKeyResponse to the protobuf v3 wire format +func (val *UpdateApiKeyResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateApiKeyResponse from the protobuf v3 wire format +func (val *UpdateApiKeyResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateApiKeyResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateApiKeyResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateApiKeyResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateApiKeyResponse + switch t := that.(type) { + case *UpdateApiKeyResponse: + that1 = t + case UpdateApiKeyResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteApiKeyRequest to the protobuf v3 wire format +func (val *DeleteApiKeyRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteApiKeyRequest from the protobuf v3 wire format +func (val *DeleteApiKeyRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteApiKeyRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteApiKeyRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteApiKeyRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteApiKeyRequest + switch t := that.(type) { + case *DeleteApiKeyRequest: + that1 = t + case DeleteApiKeyRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteApiKeyResponse to the protobuf v3 wire format +func (val *DeleteApiKeyResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteApiKeyResponse from the protobuf v3 wire format +func (val *DeleteApiKeyResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteApiKeyResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteApiKeyResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteApiKeyResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteApiKeyResponse + switch t := that.(type) { + case *DeleteApiKeyResponse: + that1 = t + case DeleteApiKeyResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetUserGroupsRequest to the protobuf v3 wire format +func (val *GetUserGroupsRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetUserGroupsRequest from the protobuf v3 wire format +func (val *GetUserGroupsRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetUserGroupsRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetUserGroupsRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetUserGroupsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetUserGroupsRequest + switch t := that.(type) { + case *GetUserGroupsRequest: + that1 = t + case GetUserGroupsRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetUserGroupsResponse to the protobuf v3 wire format +func (val *GetUserGroupsResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetUserGroupsResponse from the protobuf v3 wire format +func (val *GetUserGroupsResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetUserGroupsResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetUserGroupsResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetUserGroupsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetUserGroupsResponse + switch t := that.(type) { + case *GetUserGroupsResponse: + that1 = t + case GetUserGroupsResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetUserGroupRequest to the protobuf v3 wire format +func (val *GetUserGroupRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetUserGroupRequest from the protobuf v3 wire format +func (val *GetUserGroupRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetUserGroupRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetUserGroupRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetUserGroupRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetUserGroupRequest + switch t := that.(type) { + case *GetUserGroupRequest: + that1 = t + case GetUserGroupRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetUserGroupResponse to the protobuf v3 wire format +func (val *GetUserGroupResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetUserGroupResponse from the protobuf v3 wire format +func (val *GetUserGroupResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetUserGroupResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetUserGroupResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetUserGroupResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetUserGroupResponse + switch t := that.(type) { + case *GetUserGroupResponse: + that1 = t + case GetUserGroupResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateUserGroupRequest to the protobuf v3 wire format +func (val *CreateUserGroupRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateUserGroupRequest from the protobuf v3 wire format +func (val *CreateUserGroupRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateUserGroupRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateUserGroupRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateUserGroupRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateUserGroupRequest + switch t := that.(type) { + case *CreateUserGroupRequest: + that1 = t + case CreateUserGroupRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateUserGroupResponse to the protobuf v3 wire format +func (val *CreateUserGroupResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateUserGroupResponse from the protobuf v3 wire format +func (val *CreateUserGroupResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateUserGroupResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateUserGroupResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateUserGroupResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateUserGroupResponse + switch t := that.(type) { + case *CreateUserGroupResponse: + that1 = t + case CreateUserGroupResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateUserGroupRequest to the protobuf v3 wire format +func (val *UpdateUserGroupRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateUserGroupRequest from the protobuf v3 wire format +func (val *UpdateUserGroupRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateUserGroupRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateUserGroupRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateUserGroupRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateUserGroupRequest + switch t := that.(type) { + case *UpdateUserGroupRequest: + that1 = t + case UpdateUserGroupRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateUserGroupResponse to the protobuf v3 wire format +func (val *UpdateUserGroupResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateUserGroupResponse from the protobuf v3 wire format +func (val *UpdateUserGroupResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateUserGroupResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateUserGroupResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateUserGroupResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateUserGroupResponse + switch t := that.(type) { + case *UpdateUserGroupResponse: + that1 = t + case UpdateUserGroupResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteUserGroupRequest to the protobuf v3 wire format +func (val *DeleteUserGroupRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteUserGroupRequest from the protobuf v3 wire format +func (val *DeleteUserGroupRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteUserGroupRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteUserGroupRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteUserGroupRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteUserGroupRequest + switch t := that.(type) { + case *DeleteUserGroupRequest: + that1 = t + case DeleteUserGroupRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteUserGroupResponse to the protobuf v3 wire format +func (val *DeleteUserGroupResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteUserGroupResponse from the protobuf v3 wire format +func (val *DeleteUserGroupResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteUserGroupResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteUserGroupResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteUserGroupResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteUserGroupResponse + switch t := that.(type) { + case *DeleteUserGroupResponse: + that1 = t + case DeleteUserGroupResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SetUserGroupNamespaceAccessRequest to the protobuf v3 wire format +func (val *SetUserGroupNamespaceAccessRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SetUserGroupNamespaceAccessRequest from the protobuf v3 wire format +func (val *SetUserGroupNamespaceAccessRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SetUserGroupNamespaceAccessRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SetUserGroupNamespaceAccessRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SetUserGroupNamespaceAccessRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SetUserGroupNamespaceAccessRequest + switch t := that.(type) { + case *SetUserGroupNamespaceAccessRequest: + that1 = t + case SetUserGroupNamespaceAccessRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SetUserGroupNamespaceAccessResponse to the protobuf v3 wire format +func (val *SetUserGroupNamespaceAccessResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SetUserGroupNamespaceAccessResponse from the protobuf v3 wire format +func (val *SetUserGroupNamespaceAccessResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SetUserGroupNamespaceAccessResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SetUserGroupNamespaceAccessResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SetUserGroupNamespaceAccessResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SetUserGroupNamespaceAccessResponse + switch t := that.(type) { + case *SetUserGroupNamespaceAccessResponse: + that1 = t + case SetUserGroupNamespaceAccessResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateServiceAccountRequest to the protobuf v3 wire format +func (val *CreateServiceAccountRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateServiceAccountRequest from the protobuf v3 wire format +func (val *CreateServiceAccountRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateServiceAccountRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateServiceAccountRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateServiceAccountRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateServiceAccountRequest + switch t := that.(type) { + case *CreateServiceAccountRequest: + that1 = t + case CreateServiceAccountRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateServiceAccountResponse to the protobuf v3 wire format +func (val *CreateServiceAccountResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateServiceAccountResponse from the protobuf v3 wire format +func (val *CreateServiceAccountResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateServiceAccountResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateServiceAccountResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateServiceAccountResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateServiceAccountResponse + switch t := that.(type) { + case *CreateServiceAccountResponse: + that1 = t + case CreateServiceAccountResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetServiceAccountRequest to the protobuf v3 wire format +func (val *GetServiceAccountRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetServiceAccountRequest from the protobuf v3 wire format +func (val *GetServiceAccountRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetServiceAccountRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetServiceAccountRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetServiceAccountRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetServiceAccountRequest + switch t := that.(type) { + case *GetServiceAccountRequest: + that1 = t + case GetServiceAccountRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetServiceAccountResponse to the protobuf v3 wire format +func (val *GetServiceAccountResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetServiceAccountResponse from the protobuf v3 wire format +func (val *GetServiceAccountResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetServiceAccountResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetServiceAccountResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetServiceAccountResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetServiceAccountResponse + switch t := that.(type) { + case *GetServiceAccountResponse: + that1 = t + case GetServiceAccountResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetServiceAccountsRequest to the protobuf v3 wire format +func (val *GetServiceAccountsRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetServiceAccountsRequest from the protobuf v3 wire format +func (val *GetServiceAccountsRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetServiceAccountsRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetServiceAccountsRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetServiceAccountsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetServiceAccountsRequest + switch t := that.(type) { + case *GetServiceAccountsRequest: + that1 = t + case GetServiceAccountsRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetServiceAccountsResponse to the protobuf v3 wire format +func (val *GetServiceAccountsResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetServiceAccountsResponse from the protobuf v3 wire format +func (val *GetServiceAccountsResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetServiceAccountsResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetServiceAccountsResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetServiceAccountsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetServiceAccountsResponse + switch t := that.(type) { + case *GetServiceAccountsResponse: + that1 = t + case GetServiceAccountsResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateServiceAccountRequest to the protobuf v3 wire format +func (val *UpdateServiceAccountRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateServiceAccountRequest from the protobuf v3 wire format +func (val *UpdateServiceAccountRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateServiceAccountRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateServiceAccountRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateServiceAccountRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateServiceAccountRequest + switch t := that.(type) { + case *UpdateServiceAccountRequest: + that1 = t + case UpdateServiceAccountRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateServiceAccountResponse to the protobuf v3 wire format +func (val *UpdateServiceAccountResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateServiceAccountResponse from the protobuf v3 wire format +func (val *UpdateServiceAccountResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateServiceAccountResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateServiceAccountResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateServiceAccountResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateServiceAccountResponse + switch t := that.(type) { + case *UpdateServiceAccountResponse: + that1 = t + case UpdateServiceAccountResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteServiceAccountRequest to the protobuf v3 wire format +func (val *DeleteServiceAccountRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteServiceAccountRequest from the protobuf v3 wire format +func (val *DeleteServiceAccountRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteServiceAccountRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteServiceAccountRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteServiceAccountRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteServiceAccountRequest + switch t := that.(type) { + case *DeleteServiceAccountRequest: + that1 = t + case DeleteServiceAccountRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteServiceAccountResponse to the protobuf v3 wire format +func (val *DeleteServiceAccountResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteServiceAccountResponse from the protobuf v3 wire format +func (val *DeleteServiceAccountResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteServiceAccountResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteServiceAccountResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteServiceAccountResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteServiceAccountResponse + switch t := that.(type) { + case *DeleteServiceAccountResponse: + that1 = t + case DeleteServiceAccountResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/cloud/cloudservice/v1/request_response.pb.go b/vendor/go.temporal.io/api/cloud/cloudservice/v1/request_response.pb.go new file mode 100644 index 00000000000..8be7f7eedf9 --- /dev/null +++ b/vendor/go.temporal.io/api/cloud/cloudservice/v1/request_response.pb.go @@ -0,0 +1,5494 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/cloud/cloudservice/v1/request_response.proto + +package cloudservice + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/cloud/identity/v1" + v12 "go.temporal.io/api/cloud/namespace/v1" + v11 "go.temporal.io/api/cloud/operation/v1" + v13 "go.temporal.io/api/cloud/region/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type GetUsersRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The requested size of the page to retrieve - optional. + // Cannot exceed 1000. Defaults to 100. + PageSize int32 `protobuf:"varint,1,opt,name=page_size,json=pageSize,proto3" json:"page_size,omitempty"` + // The page token if this is continuing from another response - optional. + PageToken string `protobuf:"bytes,2,opt,name=page_token,json=pageToken,proto3" json:"page_token,omitempty"` + // Filter users by email address - optional. + Email string `protobuf:"bytes,3,opt,name=email,proto3" json:"email,omitempty"` + // Filter users by the namespace they have access to - optional. + Namespace string `protobuf:"bytes,4,opt,name=namespace,proto3" json:"namespace,omitempty"` +} + +func (x *GetUsersRequest) Reset() { + *x = GetUsersRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetUsersRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetUsersRequest) ProtoMessage() {} + +func (x *GetUsersRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetUsersRequest.ProtoReflect.Descriptor instead. +func (*GetUsersRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{0} +} + +func (x *GetUsersRequest) GetPageSize() int32 { + if x != nil { + return x.PageSize + } + return 0 +} + +func (x *GetUsersRequest) GetPageToken() string { + if x != nil { + return x.PageToken + } + return "" +} + +func (x *GetUsersRequest) GetEmail() string { + if x != nil { + return x.Email + } + return "" +} + +func (x *GetUsersRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +type GetUsersResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The list of users in ascending ids order + Users []*v1.User `protobuf:"bytes,1,rep,name=users,proto3" json:"users,omitempty"` + // The next page's token + NextPageToken string `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *GetUsersResponse) Reset() { + *x = GetUsersResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetUsersResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetUsersResponse) ProtoMessage() {} + +func (x *GetUsersResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetUsersResponse.ProtoReflect.Descriptor instead. +func (*GetUsersResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{1} +} + +func (x *GetUsersResponse) GetUsers() []*v1.User { + if x != nil { + return x.Users + } + return nil +} + +func (x *GetUsersResponse) GetNextPageToken() string { + if x != nil { + return x.NextPageToken + } + return "" +} + +type GetUserRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the user to get + UserId string `protobuf:"bytes,1,opt,name=user_id,json=userId,proto3" json:"user_id,omitempty"` +} + +func (x *GetUserRequest) Reset() { + *x = GetUserRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetUserRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetUserRequest) ProtoMessage() {} + +func (x *GetUserRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetUserRequest.ProtoReflect.Descriptor instead. +func (*GetUserRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{2} +} + +func (x *GetUserRequest) GetUserId() string { + if x != nil { + return x.UserId + } + return "" +} + +type GetUserResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The user + User *v1.User `protobuf:"bytes,1,opt,name=user,proto3" json:"user,omitempty"` +} + +func (x *GetUserResponse) Reset() { + *x = GetUserResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetUserResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetUserResponse) ProtoMessage() {} + +func (x *GetUserResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetUserResponse.ProtoReflect.Descriptor instead. +func (*GetUserResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{3} +} + +func (x *GetUserResponse) GetUser() *v1.User { + if x != nil { + return x.User + } + return nil +} + +type CreateUserRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The spec for the user to invite + Spec *v1.UserSpec `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"` + // The id to use for this async operation - optional + AsyncOperationId string `protobuf:"bytes,2,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *CreateUserRequest) Reset() { + *x = CreateUserRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateUserRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateUserRequest) ProtoMessage() {} + +func (x *CreateUserRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateUserRequest.ProtoReflect.Descriptor instead. +func (*CreateUserRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{4} +} + +func (x *CreateUserRequest) GetSpec() *v1.UserSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *CreateUserRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type CreateUserResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the user that was invited + UserId string `protobuf:"bytes,1,opt,name=user_id,json=userId,proto3" json:"user_id,omitempty"` + // The async operation + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,2,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *CreateUserResponse) Reset() { + *x = CreateUserResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateUserResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateUserResponse) ProtoMessage() {} + +func (x *CreateUserResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateUserResponse.ProtoReflect.Descriptor instead. +func (*CreateUserResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{5} +} + +func (x *CreateUserResponse) GetUserId() string { + if x != nil { + return x.UserId + } + return "" +} + +func (x *CreateUserResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type UpdateUserRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the user to update + UserId string `protobuf:"bytes,1,opt,name=user_id,json=userId,proto3" json:"user_id,omitempty"` + // The new user specification + Spec *v1.UserSpec `protobuf:"bytes,2,opt,name=spec,proto3" json:"spec,omitempty"` + // The version of the user for which this update is intended for + // The latest version can be found in the GetUser operation response + ResourceVersion string `protobuf:"bytes,3,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The id to use for this async operation - optional + AsyncOperationId string `protobuf:"bytes,4,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *UpdateUserRequest) Reset() { + *x = UpdateUserRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateUserRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateUserRequest) ProtoMessage() {} + +func (x *UpdateUserRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateUserRequest.ProtoReflect.Descriptor instead. +func (*UpdateUserRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{6} +} + +func (x *UpdateUserRequest) GetUserId() string { + if x != nil { + return x.UserId + } + return "" +} + +func (x *UpdateUserRequest) GetSpec() *v1.UserSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *UpdateUserRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *UpdateUserRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type UpdateUserResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *UpdateUserResponse) Reset() { + *x = UpdateUserResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateUserResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateUserResponse) ProtoMessage() {} + +func (x *UpdateUserResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateUserResponse.ProtoReflect.Descriptor instead. +func (*UpdateUserResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{7} +} + +func (x *UpdateUserResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type DeleteUserRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the user to delete + UserId string `protobuf:"bytes,1,opt,name=user_id,json=userId,proto3" json:"user_id,omitempty"` + // The version of the user for which this delete is intended for + // The latest version can be found in the GetUser operation response + ResourceVersion string `protobuf:"bytes,2,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The id to use for this async operation - optional + AsyncOperationId string `protobuf:"bytes,3,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *DeleteUserRequest) Reset() { + *x = DeleteUserRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteUserRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteUserRequest) ProtoMessage() {} + +func (x *DeleteUserRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteUserRequest.ProtoReflect.Descriptor instead. +func (*DeleteUserRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{8} +} + +func (x *DeleteUserRequest) GetUserId() string { + if x != nil { + return x.UserId + } + return "" +} + +func (x *DeleteUserRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *DeleteUserRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type DeleteUserResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *DeleteUserResponse) Reset() { + *x = DeleteUserResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteUserResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteUserResponse) ProtoMessage() {} + +func (x *DeleteUserResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteUserResponse.ProtoReflect.Descriptor instead. +func (*DeleteUserResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{9} +} + +func (x *DeleteUserResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type SetUserNamespaceAccessRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace to set permissions for + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The id of the user to set permissions for + UserId string `protobuf:"bytes,2,opt,name=user_id,json=userId,proto3" json:"user_id,omitempty"` + // The namespace access to assign the user + Access *v1.NamespaceAccess `protobuf:"bytes,3,opt,name=access,proto3" json:"access,omitempty"` + // The version of the user for which this update is intended for + // The latest version can be found in the GetUser operation response + ResourceVersion string `protobuf:"bytes,4,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The id to use for this async operation - optional + AsyncOperationId string `protobuf:"bytes,5,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *SetUserNamespaceAccessRequest) Reset() { + *x = SetUserNamespaceAccessRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SetUserNamespaceAccessRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SetUserNamespaceAccessRequest) ProtoMessage() {} + +func (x *SetUserNamespaceAccessRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SetUserNamespaceAccessRequest.ProtoReflect.Descriptor instead. +func (*SetUserNamespaceAccessRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{10} +} + +func (x *SetUserNamespaceAccessRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *SetUserNamespaceAccessRequest) GetUserId() string { + if x != nil { + return x.UserId + } + return "" +} + +func (x *SetUserNamespaceAccessRequest) GetAccess() *v1.NamespaceAccess { + if x != nil { + return x.Access + } + return nil +} + +func (x *SetUserNamespaceAccessRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *SetUserNamespaceAccessRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type SetUserNamespaceAccessResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *SetUserNamespaceAccessResponse) Reset() { + *x = SetUserNamespaceAccessResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SetUserNamespaceAccessResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SetUserNamespaceAccessResponse) ProtoMessage() {} + +func (x *SetUserNamespaceAccessResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SetUserNamespaceAccessResponse.ProtoReflect.Descriptor instead. +func (*SetUserNamespaceAccessResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{11} +} + +func (x *SetUserNamespaceAccessResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type GetAsyncOperationRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the async operation to get + AsyncOperationId string `protobuf:"bytes,1,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *GetAsyncOperationRequest) Reset() { + *x = GetAsyncOperationRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetAsyncOperationRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetAsyncOperationRequest) ProtoMessage() {} + +func (x *GetAsyncOperationRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetAsyncOperationRequest.ProtoReflect.Descriptor instead. +func (*GetAsyncOperationRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{12} +} + +func (x *GetAsyncOperationRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type GetAsyncOperationResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *GetAsyncOperationResponse) Reset() { + *x = GetAsyncOperationResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetAsyncOperationResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetAsyncOperationResponse) ProtoMessage() {} + +func (x *GetAsyncOperationResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetAsyncOperationResponse.ProtoReflect.Descriptor instead. +func (*GetAsyncOperationResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{13} +} + +func (x *GetAsyncOperationResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type CreateNamespaceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace specification. + Spec *v12.NamespaceSpec `protobuf:"bytes,2,opt,name=spec,proto3" json:"spec,omitempty"` + // The id to use for this async operation. + // Optional, if not provided a random id will be generated. + AsyncOperationId string `protobuf:"bytes,3,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *CreateNamespaceRequest) Reset() { + *x = CreateNamespaceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateNamespaceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateNamespaceRequest) ProtoMessage() {} + +func (x *CreateNamespaceRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateNamespaceRequest.ProtoReflect.Descriptor instead. +func (*CreateNamespaceRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{14} +} + +func (x *CreateNamespaceRequest) GetSpec() *v12.NamespaceSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *CreateNamespaceRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type CreateNamespaceResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace that was created. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,2,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *CreateNamespaceResponse) Reset() { + *x = CreateNamespaceResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateNamespaceResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateNamespaceResponse) ProtoMessage() {} + +func (x *CreateNamespaceResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[15] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateNamespaceResponse.ProtoReflect.Descriptor instead. +func (*CreateNamespaceResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{15} +} + +func (x *CreateNamespaceResponse) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *CreateNamespaceResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type GetNamespacesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The requested size of the page to retrieve. + // Cannot exceed 1000. + // Optional, defaults to 100. + PageSize int32 `protobuf:"varint,1,opt,name=page_size,json=pageSize,proto3" json:"page_size,omitempty"` + // The page token if this is continuing from another response. + // Optional, defaults to empty. + PageToken string `protobuf:"bytes,2,opt,name=page_token,json=pageToken,proto3" json:"page_token,omitempty"` + // Filter namespaces by their name. + // Optional, defaults to empty. + Name string `protobuf:"bytes,3,opt,name=name,proto3" json:"name,omitempty"` +} + +func (x *GetNamespacesRequest) Reset() { + *x = GetNamespacesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetNamespacesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetNamespacesRequest) ProtoMessage() {} + +func (x *GetNamespacesRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[16] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetNamespacesRequest.ProtoReflect.Descriptor instead. +func (*GetNamespacesRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{16} +} + +func (x *GetNamespacesRequest) GetPageSize() int32 { + if x != nil { + return x.PageSize + } + return 0 +} + +func (x *GetNamespacesRequest) GetPageToken() string { + if x != nil { + return x.PageToken + } + return "" +} + +func (x *GetNamespacesRequest) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type GetNamespacesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The list of namespaces in ascending name order. + Namespaces []*v12.Namespace `protobuf:"bytes,1,rep,name=namespaces,proto3" json:"namespaces,omitempty"` + // The next page's token. + NextPageToken string `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *GetNamespacesResponse) Reset() { + *x = GetNamespacesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetNamespacesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetNamespacesResponse) ProtoMessage() {} + +func (x *GetNamespacesResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[17] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetNamespacesResponse.ProtoReflect.Descriptor instead. +func (*GetNamespacesResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{17} +} + +func (x *GetNamespacesResponse) GetNamespaces() []*v12.Namespace { + if x != nil { + return x.Namespaces + } + return nil +} + +func (x *GetNamespacesResponse) GetNextPageToken() string { + if x != nil { + return x.NextPageToken + } + return "" +} + +type GetNamespaceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace to get. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` +} + +func (x *GetNamespaceRequest) Reset() { + *x = GetNamespaceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetNamespaceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetNamespaceRequest) ProtoMessage() {} + +func (x *GetNamespaceRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[18] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetNamespaceRequest.ProtoReflect.Descriptor instead. +func (*GetNamespaceRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{18} +} + +func (x *GetNamespaceRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +type GetNamespaceResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace. + Namespace *v12.Namespace `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` +} + +func (x *GetNamespaceResponse) Reset() { + *x = GetNamespaceResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetNamespaceResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetNamespaceResponse) ProtoMessage() {} + +func (x *GetNamespaceResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[19] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetNamespaceResponse.ProtoReflect.Descriptor instead. +func (*GetNamespaceResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{19} +} + +func (x *GetNamespaceResponse) GetNamespace() *v12.Namespace { + if x != nil { + return x.Namespace + } + return nil +} + +type UpdateNamespaceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace to update. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The new namespace specification. + Spec *v12.NamespaceSpec `protobuf:"bytes,2,opt,name=spec,proto3" json:"spec,omitempty"` + // The version of the namespace for which this update is intended for. + // The latest version can be found in the namespace status. + ResourceVersion string `protobuf:"bytes,3,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The id to use for this async operation. + // Optional, if not provided a random id will be generated. + AsyncOperationId string `protobuf:"bytes,4,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *UpdateNamespaceRequest) Reset() { + *x = UpdateNamespaceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateNamespaceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateNamespaceRequest) ProtoMessage() {} + +func (x *UpdateNamespaceRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[20] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateNamespaceRequest.ProtoReflect.Descriptor instead. +func (*UpdateNamespaceRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{20} +} + +func (x *UpdateNamespaceRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *UpdateNamespaceRequest) GetSpec() *v12.NamespaceSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *UpdateNamespaceRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *UpdateNamespaceRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type UpdateNamespaceResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *UpdateNamespaceResponse) Reset() { + *x = UpdateNamespaceResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateNamespaceResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateNamespaceResponse) ProtoMessage() {} + +func (x *UpdateNamespaceResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[21] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateNamespaceResponse.ProtoReflect.Descriptor instead. +func (*UpdateNamespaceResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{21} +} + +func (x *UpdateNamespaceResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type RenameCustomSearchAttributeRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace to rename the custom search attribute for. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The existing name of the custom search attribute to be renamed. + ExistingCustomSearchAttributeName string `protobuf:"bytes,2,opt,name=existing_custom_search_attribute_name,json=existingCustomSearchAttributeName,proto3" json:"existing_custom_search_attribute_name,omitempty"` + // The new name of the custom search attribute. + NewCustomSearchAttributeName string `protobuf:"bytes,3,opt,name=new_custom_search_attribute_name,json=newCustomSearchAttributeName,proto3" json:"new_custom_search_attribute_name,omitempty"` + // The version of the namespace for which this update is intended for. + // The latest version can be found in the namespace status. + ResourceVersion string `protobuf:"bytes,4,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The id to use for this async operation. + // Optional, if not provided a random id will be generated. + AsyncOperationId string `protobuf:"bytes,5,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *RenameCustomSearchAttributeRequest) Reset() { + *x = RenameCustomSearchAttributeRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RenameCustomSearchAttributeRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RenameCustomSearchAttributeRequest) ProtoMessage() {} + +func (x *RenameCustomSearchAttributeRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[22] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RenameCustomSearchAttributeRequest.ProtoReflect.Descriptor instead. +func (*RenameCustomSearchAttributeRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{22} +} + +func (x *RenameCustomSearchAttributeRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RenameCustomSearchAttributeRequest) GetExistingCustomSearchAttributeName() string { + if x != nil { + return x.ExistingCustomSearchAttributeName + } + return "" +} + +func (x *RenameCustomSearchAttributeRequest) GetNewCustomSearchAttributeName() string { + if x != nil { + return x.NewCustomSearchAttributeName + } + return "" +} + +func (x *RenameCustomSearchAttributeRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *RenameCustomSearchAttributeRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type RenameCustomSearchAttributeResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *RenameCustomSearchAttributeResponse) Reset() { + *x = RenameCustomSearchAttributeResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RenameCustomSearchAttributeResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RenameCustomSearchAttributeResponse) ProtoMessage() {} + +func (x *RenameCustomSearchAttributeResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[23] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RenameCustomSearchAttributeResponse.ProtoReflect.Descriptor instead. +func (*RenameCustomSearchAttributeResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{23} +} + +func (x *RenameCustomSearchAttributeResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type DeleteNamespaceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace to delete. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The version of the namespace for which this delete is intended for. + // The latest version can be found in the namespace status. + ResourceVersion string `protobuf:"bytes,2,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The id to use for this async operation. + // Optional, if not provided a random id will be generated. + AsyncOperationId string `protobuf:"bytes,3,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *DeleteNamespaceRequest) Reset() { + *x = DeleteNamespaceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteNamespaceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteNamespaceRequest) ProtoMessage() {} + +func (x *DeleteNamespaceRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[24] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteNamespaceRequest.ProtoReflect.Descriptor instead. +func (*DeleteNamespaceRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{24} +} + +func (x *DeleteNamespaceRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *DeleteNamespaceRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *DeleteNamespaceRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type DeleteNamespaceResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *DeleteNamespaceResponse) Reset() { + *x = DeleteNamespaceResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteNamespaceResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteNamespaceResponse) ProtoMessage() {} + +func (x *DeleteNamespaceResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[25] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteNamespaceResponse.ProtoReflect.Descriptor instead. +func (*DeleteNamespaceResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{25} +} + +func (x *DeleteNamespaceResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type FailoverNamespaceRegionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace to failover. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The id of the region to failover to. + // Must be a region that the namespace is currently available in. + Region string `protobuf:"bytes,2,opt,name=region,proto3" json:"region,omitempty"` + // The id to use for this async operation - optional. + AsyncOperationId string `protobuf:"bytes,3,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *FailoverNamespaceRegionRequest) Reset() { + *x = FailoverNamespaceRegionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FailoverNamespaceRegionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FailoverNamespaceRegionRequest) ProtoMessage() {} + +func (x *FailoverNamespaceRegionRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[26] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FailoverNamespaceRegionRequest.ProtoReflect.Descriptor instead. +func (*FailoverNamespaceRegionRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{26} +} + +func (x *FailoverNamespaceRegionRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *FailoverNamespaceRegionRequest) GetRegion() string { + if x != nil { + return x.Region + } + return "" +} + +func (x *FailoverNamespaceRegionRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type FailoverNamespaceRegionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *FailoverNamespaceRegionResponse) Reset() { + *x = FailoverNamespaceRegionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FailoverNamespaceRegionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FailoverNamespaceRegionResponse) ProtoMessage() {} + +func (x *FailoverNamespaceRegionResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[27] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FailoverNamespaceRegionResponse.ProtoReflect.Descriptor instead. +func (*FailoverNamespaceRegionResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{27} +} + +func (x *FailoverNamespaceRegionResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type AddNamespaceRegionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace to add the region to. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The id of the standby region to add to the namespace. + // The GetRegions API can be used to get the list of valid region ids. + // Example: "aws-us-west-2". + Region string `protobuf:"bytes,2,opt,name=region,proto3" json:"region,omitempty"` + // The version of the namespace for which this add region operation is intended for. + // The latest version can be found in the GetNamespace operation response. + ResourceVersion string `protobuf:"bytes,3,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The id to use for this async operation - optional. + AsyncOperationId string `protobuf:"bytes,4,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *AddNamespaceRegionRequest) Reset() { + *x = AddNamespaceRegionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AddNamespaceRegionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AddNamespaceRegionRequest) ProtoMessage() {} + +func (x *AddNamespaceRegionRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[28] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AddNamespaceRegionRequest.ProtoReflect.Descriptor instead. +func (*AddNamespaceRegionRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{28} +} + +func (x *AddNamespaceRegionRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *AddNamespaceRegionRequest) GetRegion() string { + if x != nil { + return x.Region + } + return "" +} + +func (x *AddNamespaceRegionRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *AddNamespaceRegionRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type AddNamespaceRegionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *AddNamespaceRegionResponse) Reset() { + *x = AddNamespaceRegionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AddNamespaceRegionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AddNamespaceRegionResponse) ProtoMessage() {} + +func (x *AddNamespaceRegionResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[29] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AddNamespaceRegionResponse.ProtoReflect.Descriptor instead. +func (*AddNamespaceRegionResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{29} +} + +func (x *AddNamespaceRegionResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type GetRegionsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *GetRegionsRequest) Reset() { + *x = GetRegionsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetRegionsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetRegionsRequest) ProtoMessage() {} + +func (x *GetRegionsRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[30] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetRegionsRequest.ProtoReflect.Descriptor instead. +func (*GetRegionsRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{30} +} + +type GetRegionsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The temporal cloud regions. + Regions []*v13.Region `protobuf:"bytes,1,rep,name=regions,proto3" json:"regions,omitempty"` +} + +func (x *GetRegionsResponse) Reset() { + *x = GetRegionsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetRegionsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetRegionsResponse) ProtoMessage() {} + +func (x *GetRegionsResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[31] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetRegionsResponse.ProtoReflect.Descriptor instead. +func (*GetRegionsResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{31} +} + +func (x *GetRegionsResponse) GetRegions() []*v13.Region { + if x != nil { + return x.Regions + } + return nil +} + +type GetRegionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the region to get. + Region string `protobuf:"bytes,1,opt,name=region,proto3" json:"region,omitempty"` +} + +func (x *GetRegionRequest) Reset() { + *x = GetRegionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetRegionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetRegionRequest) ProtoMessage() {} + +func (x *GetRegionRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[32] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetRegionRequest.ProtoReflect.Descriptor instead. +func (*GetRegionRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{32} +} + +func (x *GetRegionRequest) GetRegion() string { + if x != nil { + return x.Region + } + return "" +} + +type GetRegionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The temporal cloud region. + Region *v13.Region `protobuf:"bytes,1,opt,name=region,proto3" json:"region,omitempty"` +} + +func (x *GetRegionResponse) Reset() { + *x = GetRegionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetRegionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetRegionResponse) ProtoMessage() {} + +func (x *GetRegionResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[33] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetRegionResponse.ProtoReflect.Descriptor instead. +func (*GetRegionResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{33} +} + +func (x *GetRegionResponse) GetRegion() *v13.Region { + if x != nil { + return x.Region + } + return nil +} + +type GetApiKeysRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The requested size of the page to retrieve - optional. + // Cannot exceed 1000. Defaults to 100. + PageSize int32 `protobuf:"varint,1,opt,name=page_size,json=pageSize,proto3" json:"page_size,omitempty"` + // The page token if this is continuing from another response - optional. + PageToken string `protobuf:"bytes,2,opt,name=page_token,json=pageToken,proto3" json:"page_token,omitempty"` + // Filter api keys by owner id - optional. + OwnerId string `protobuf:"bytes,3,opt,name=owner_id,json=ownerId,proto3" json:"owner_id,omitempty"` + // Filter api keys by owner type - optional. + // Possible values: user, service-account + OwnerType string `protobuf:"bytes,4,opt,name=owner_type,json=ownerType,proto3" json:"owner_type,omitempty"` +} + +func (x *GetApiKeysRequest) Reset() { + *x = GetApiKeysRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetApiKeysRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetApiKeysRequest) ProtoMessage() {} + +func (x *GetApiKeysRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[34] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetApiKeysRequest.ProtoReflect.Descriptor instead. +func (*GetApiKeysRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{34} +} + +func (x *GetApiKeysRequest) GetPageSize() int32 { + if x != nil { + return x.PageSize + } + return 0 +} + +func (x *GetApiKeysRequest) GetPageToken() string { + if x != nil { + return x.PageToken + } + return "" +} + +func (x *GetApiKeysRequest) GetOwnerId() string { + if x != nil { + return x.OwnerId + } + return "" +} + +func (x *GetApiKeysRequest) GetOwnerType() string { + if x != nil { + return x.OwnerType + } + return "" +} + +type GetApiKeysResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The list of api keys in ascending id order. + ApiKeys []*v1.ApiKey `protobuf:"bytes,1,rep,name=api_keys,json=apiKeys,proto3" json:"api_keys,omitempty"` + // The next page's token. + NextPageToken string `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *GetApiKeysResponse) Reset() { + *x = GetApiKeysResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetApiKeysResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetApiKeysResponse) ProtoMessage() {} + +func (x *GetApiKeysResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[35] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetApiKeysResponse.ProtoReflect.Descriptor instead. +func (*GetApiKeysResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{35} +} + +func (x *GetApiKeysResponse) GetApiKeys() []*v1.ApiKey { + if x != nil { + return x.ApiKeys + } + return nil +} + +func (x *GetApiKeysResponse) GetNextPageToken() string { + if x != nil { + return x.NextPageToken + } + return "" +} + +type GetApiKeyRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the api key to get. + KeyId string `protobuf:"bytes,1,opt,name=key_id,json=keyId,proto3" json:"key_id,omitempty"` +} + +func (x *GetApiKeyRequest) Reset() { + *x = GetApiKeyRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetApiKeyRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetApiKeyRequest) ProtoMessage() {} + +func (x *GetApiKeyRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[36] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetApiKeyRequest.ProtoReflect.Descriptor instead. +func (*GetApiKeyRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{36} +} + +func (x *GetApiKeyRequest) GetKeyId() string { + if x != nil { + return x.KeyId + } + return "" +} + +type GetApiKeyResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The api key. + ApiKey *v1.ApiKey `protobuf:"bytes,1,opt,name=api_key,json=apiKey,proto3" json:"api_key,omitempty"` +} + +func (x *GetApiKeyResponse) Reset() { + *x = GetApiKeyResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetApiKeyResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetApiKeyResponse) ProtoMessage() {} + +func (x *GetApiKeyResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[37] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetApiKeyResponse.ProtoReflect.Descriptor instead. +func (*GetApiKeyResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{37} +} + +func (x *GetApiKeyResponse) GetApiKey() *v1.ApiKey { + if x != nil { + return x.ApiKey + } + return nil +} + +type CreateApiKeyRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The spec for the api key to create. + // Create api key only supports service-account owner type for now. + Spec *v1.ApiKeySpec `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"` + // The id to use for this async operation - optional. + AsyncOperationId string `protobuf:"bytes,2,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *CreateApiKeyRequest) Reset() { + *x = CreateApiKeyRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateApiKeyRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateApiKeyRequest) ProtoMessage() {} + +func (x *CreateApiKeyRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[38] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateApiKeyRequest.ProtoReflect.Descriptor instead. +func (*CreateApiKeyRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{38} +} + +func (x *CreateApiKeyRequest) GetSpec() *v1.ApiKeySpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *CreateApiKeyRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type CreateApiKeyResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the api key created. + KeyId string `protobuf:"bytes,1,opt,name=key_id,json=keyId,proto3" json:"key_id,omitempty"` + // The token of the api key created. + // This is a secret and should be stored securely. + // It will not be retrievable after this response. + Token string `protobuf:"bytes,2,opt,name=token,proto3" json:"token,omitempty"` + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,3,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *CreateApiKeyResponse) Reset() { + *x = CreateApiKeyResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateApiKeyResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateApiKeyResponse) ProtoMessage() {} + +func (x *CreateApiKeyResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[39] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateApiKeyResponse.ProtoReflect.Descriptor instead. +func (*CreateApiKeyResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{39} +} + +func (x *CreateApiKeyResponse) GetKeyId() string { + if x != nil { + return x.KeyId + } + return "" +} + +func (x *CreateApiKeyResponse) GetToken() string { + if x != nil { + return x.Token + } + return "" +} + +func (x *CreateApiKeyResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type UpdateApiKeyRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the api key to update. + KeyId string `protobuf:"bytes,1,opt,name=key_id,json=keyId,proto3" json:"key_id,omitempty"` + // The new api key specification. + Spec *v1.ApiKeySpec `protobuf:"bytes,2,opt,name=spec,proto3" json:"spec,omitempty"` + // The version of the api key for which this update is intended for. + // The latest version can be found in the GetApiKey operation response. + ResourceVersion string `protobuf:"bytes,3,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The id to use for this async operation - optional. + AsyncOperationId string `protobuf:"bytes,4,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *UpdateApiKeyRequest) Reset() { + *x = UpdateApiKeyRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateApiKeyRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateApiKeyRequest) ProtoMessage() {} + +func (x *UpdateApiKeyRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[40] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateApiKeyRequest.ProtoReflect.Descriptor instead. +func (*UpdateApiKeyRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{40} +} + +func (x *UpdateApiKeyRequest) GetKeyId() string { + if x != nil { + return x.KeyId + } + return "" +} + +func (x *UpdateApiKeyRequest) GetSpec() *v1.ApiKeySpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *UpdateApiKeyRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *UpdateApiKeyRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type UpdateApiKeyResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *UpdateApiKeyResponse) Reset() { + *x = UpdateApiKeyResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateApiKeyResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateApiKeyResponse) ProtoMessage() {} + +func (x *UpdateApiKeyResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[41] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateApiKeyResponse.ProtoReflect.Descriptor instead. +func (*UpdateApiKeyResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{41} +} + +func (x *UpdateApiKeyResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type DeleteApiKeyRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the api key to delete. + KeyId string `protobuf:"bytes,1,opt,name=key_id,json=keyId,proto3" json:"key_id,omitempty"` + // The version of the api key for which this delete is intended for. + // The latest version can be found in the GetApiKey operation response. + ResourceVersion string `protobuf:"bytes,2,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The id to use for this async operation - optional. + AsyncOperationId string `protobuf:"bytes,3,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *DeleteApiKeyRequest) Reset() { + *x = DeleteApiKeyRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteApiKeyRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteApiKeyRequest) ProtoMessage() {} + +func (x *DeleteApiKeyRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[42] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteApiKeyRequest.ProtoReflect.Descriptor instead. +func (*DeleteApiKeyRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{42} +} + +func (x *DeleteApiKeyRequest) GetKeyId() string { + if x != nil { + return x.KeyId + } + return "" +} + +func (x *DeleteApiKeyRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *DeleteApiKeyRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type DeleteApiKeyResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *DeleteApiKeyResponse) Reset() { + *x = DeleteApiKeyResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteApiKeyResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteApiKeyResponse) ProtoMessage() {} + +func (x *DeleteApiKeyResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[43] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteApiKeyResponse.ProtoReflect.Descriptor instead. +func (*DeleteApiKeyResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{43} +} + +func (x *DeleteApiKeyResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type GetUserGroupsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The requested size of the page to retrieve - optional. + // Cannot exceed 1000. Defaults to 100. + PageSize int32 `protobuf:"varint,1,opt,name=page_size,json=pageSize,proto3" json:"page_size,omitempty"` + // The page token if this is continuing from another response - optional. + PageToken string `protobuf:"bytes,2,opt,name=page_token,json=pageToken,proto3" json:"page_token,omitempty"` + // Filter groups by the namespace they have access to - optional. + Namespace string `protobuf:"bytes,3,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Filter groups by the display name - optional. + DisplayName string `protobuf:"bytes,4,opt,name=display_name,json=displayName,proto3" json:"display_name,omitempty"` + // Filter groups by the google group specification - optional. + GoogleGroup *GetUserGroupsRequest_GoogleGroupFilter `protobuf:"bytes,5,opt,name=google_group,json=googleGroup,proto3" json:"google_group,omitempty"` +} + +func (x *GetUserGroupsRequest) Reset() { + *x = GetUserGroupsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetUserGroupsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetUserGroupsRequest) ProtoMessage() {} + +func (x *GetUserGroupsRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[44] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetUserGroupsRequest.ProtoReflect.Descriptor instead. +func (*GetUserGroupsRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{44} +} + +func (x *GetUserGroupsRequest) GetPageSize() int32 { + if x != nil { + return x.PageSize + } + return 0 +} + +func (x *GetUserGroupsRequest) GetPageToken() string { + if x != nil { + return x.PageToken + } + return "" +} + +func (x *GetUserGroupsRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *GetUserGroupsRequest) GetDisplayName() string { + if x != nil { + return x.DisplayName + } + return "" +} + +func (x *GetUserGroupsRequest) GetGoogleGroup() *GetUserGroupsRequest_GoogleGroupFilter { + if x != nil { + return x.GoogleGroup + } + return nil +} + +type GetUserGroupsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The list of groups in ascending name order. + Groups []*v1.UserGroup `protobuf:"bytes,1,rep,name=groups,proto3" json:"groups,omitempty"` + // The next page's token. + NextPageToken string `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *GetUserGroupsResponse) Reset() { + *x = GetUserGroupsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetUserGroupsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetUserGroupsResponse) ProtoMessage() {} + +func (x *GetUserGroupsResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[45] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetUserGroupsResponse.ProtoReflect.Descriptor instead. +func (*GetUserGroupsResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{45} +} + +func (x *GetUserGroupsResponse) GetGroups() []*v1.UserGroup { + if x != nil { + return x.Groups + } + return nil +} + +func (x *GetUserGroupsResponse) GetNextPageToken() string { + if x != nil { + return x.NextPageToken + } + return "" +} + +type GetUserGroupRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the group to get. + GroupId string `protobuf:"bytes,1,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"` +} + +func (x *GetUserGroupRequest) Reset() { + *x = GetUserGroupRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[46] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetUserGroupRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetUserGroupRequest) ProtoMessage() {} + +func (x *GetUserGroupRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[46] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetUserGroupRequest.ProtoReflect.Descriptor instead. +func (*GetUserGroupRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{46} +} + +func (x *GetUserGroupRequest) GetGroupId() string { + if x != nil { + return x.GroupId + } + return "" +} + +type GetUserGroupResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The group. + Group *v1.UserGroup `protobuf:"bytes,1,opt,name=group,proto3" json:"group,omitempty"` +} + +func (x *GetUserGroupResponse) Reset() { + *x = GetUserGroupResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[47] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetUserGroupResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetUserGroupResponse) ProtoMessage() {} + +func (x *GetUserGroupResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[47] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetUserGroupResponse.ProtoReflect.Descriptor instead. +func (*GetUserGroupResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{47} +} + +func (x *GetUserGroupResponse) GetGroup() *v1.UserGroup { + if x != nil { + return x.Group + } + return nil +} + +type CreateUserGroupRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The spec for the group to create. + Spec *v1.UserGroupSpec `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"` + // The id to use for this async operation. + // Optional, if not provided a random id will be generated. + AsyncOperationId string `protobuf:"bytes,2,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *CreateUserGroupRequest) Reset() { + *x = CreateUserGroupRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[48] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateUserGroupRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateUserGroupRequest) ProtoMessage() {} + +func (x *CreateUserGroupRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[48] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateUserGroupRequest.ProtoReflect.Descriptor instead. +func (*CreateUserGroupRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{48} +} + +func (x *CreateUserGroupRequest) GetSpec() *v1.UserGroupSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *CreateUserGroupRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type CreateUserGroupResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the group that was created. + GroupId string `protobuf:"bytes,1,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"` + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,2,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *CreateUserGroupResponse) Reset() { + *x = CreateUserGroupResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[49] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateUserGroupResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateUserGroupResponse) ProtoMessage() {} + +func (x *CreateUserGroupResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[49] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateUserGroupResponse.ProtoReflect.Descriptor instead. +func (*CreateUserGroupResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{49} +} + +func (x *CreateUserGroupResponse) GetGroupId() string { + if x != nil { + return x.GroupId + } + return "" +} + +func (x *CreateUserGroupResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type UpdateUserGroupRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the group to update. + GroupId string `protobuf:"bytes,1,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"` + // The new group specification. + Spec *v1.UserGroupSpec `protobuf:"bytes,2,opt,name=spec,proto3" json:"spec,omitempty"` + // The version of the group for which this update is intended for. + // The latest version can be found in the GetGroup operation response. + ResourceVersion string `protobuf:"bytes,3,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The id to use for this async operation. + // Optional, if not provided a random id will be generated. + AsyncOperationId string `protobuf:"bytes,4,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *UpdateUserGroupRequest) Reset() { + *x = UpdateUserGroupRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[50] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateUserGroupRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateUserGroupRequest) ProtoMessage() {} + +func (x *UpdateUserGroupRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[50] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateUserGroupRequest.ProtoReflect.Descriptor instead. +func (*UpdateUserGroupRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{50} +} + +func (x *UpdateUserGroupRequest) GetGroupId() string { + if x != nil { + return x.GroupId + } + return "" +} + +func (x *UpdateUserGroupRequest) GetSpec() *v1.UserGroupSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *UpdateUserGroupRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *UpdateUserGroupRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type UpdateUserGroupResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *UpdateUserGroupResponse) Reset() { + *x = UpdateUserGroupResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[51] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateUserGroupResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateUserGroupResponse) ProtoMessage() {} + +func (x *UpdateUserGroupResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[51] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateUserGroupResponse.ProtoReflect.Descriptor instead. +func (*UpdateUserGroupResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{51} +} + +func (x *UpdateUserGroupResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type DeleteUserGroupRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the group to delete. + GroupId string `protobuf:"bytes,1,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"` + // The version of the group for which this delete is intended for. + // The latest version can be found in the GetGroup operation response. + ResourceVersion string `protobuf:"bytes,2,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The id to use for this async operation. + // Optional, if not provided a random id will be generated. + AsyncOperationId string `protobuf:"bytes,3,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *DeleteUserGroupRequest) Reset() { + *x = DeleteUserGroupRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[52] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteUserGroupRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteUserGroupRequest) ProtoMessage() {} + +func (x *DeleteUserGroupRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[52] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteUserGroupRequest.ProtoReflect.Descriptor instead. +func (*DeleteUserGroupRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{52} +} + +func (x *DeleteUserGroupRequest) GetGroupId() string { + if x != nil { + return x.GroupId + } + return "" +} + +func (x *DeleteUserGroupRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *DeleteUserGroupRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type DeleteUserGroupResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *DeleteUserGroupResponse) Reset() { + *x = DeleteUserGroupResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[53] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteUserGroupResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteUserGroupResponse) ProtoMessage() {} + +func (x *DeleteUserGroupResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[53] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteUserGroupResponse.ProtoReflect.Descriptor instead. +func (*DeleteUserGroupResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{53} +} + +func (x *DeleteUserGroupResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type SetUserGroupNamespaceAccessRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace to set permissions for. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The id of the group to set permissions for. + GroupId string `protobuf:"bytes,2,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"` + // The namespace access to assign the group. If left empty, the group will be removed from the namespace access. + Access *v1.NamespaceAccess `protobuf:"bytes,3,opt,name=access,proto3" json:"access,omitempty"` + // The version of the group for which this update is intended for. + // The latest version can be found in the GetGroup operation response. + ResourceVersion string `protobuf:"bytes,4,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The id to use for this async operation - optional. + AsyncOperationId string `protobuf:"bytes,5,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *SetUserGroupNamespaceAccessRequest) Reset() { + *x = SetUserGroupNamespaceAccessRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[54] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SetUserGroupNamespaceAccessRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SetUserGroupNamespaceAccessRequest) ProtoMessage() {} + +func (x *SetUserGroupNamespaceAccessRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[54] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SetUserGroupNamespaceAccessRequest.ProtoReflect.Descriptor instead. +func (*SetUserGroupNamespaceAccessRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{54} +} + +func (x *SetUserGroupNamespaceAccessRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *SetUserGroupNamespaceAccessRequest) GetGroupId() string { + if x != nil { + return x.GroupId + } + return "" +} + +func (x *SetUserGroupNamespaceAccessRequest) GetAccess() *v1.NamespaceAccess { + if x != nil { + return x.Access + } + return nil +} + +func (x *SetUserGroupNamespaceAccessRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *SetUserGroupNamespaceAccessRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type SetUserGroupNamespaceAccessResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *SetUserGroupNamespaceAccessResponse) Reset() { + *x = SetUserGroupNamespaceAccessResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[55] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SetUserGroupNamespaceAccessResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SetUserGroupNamespaceAccessResponse) ProtoMessage() {} + +func (x *SetUserGroupNamespaceAccessResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[55] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SetUserGroupNamespaceAccessResponse.ProtoReflect.Descriptor instead. +func (*SetUserGroupNamespaceAccessResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{55} +} + +func (x *SetUserGroupNamespaceAccessResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type CreateServiceAccountRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The spec of the service account to create. + Spec *v1.ServiceAccountSpec `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"` + // The ID to use for this async operation - optional. + AsyncOperationId string `protobuf:"bytes,2,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *CreateServiceAccountRequest) Reset() { + *x = CreateServiceAccountRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[56] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateServiceAccountRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateServiceAccountRequest) ProtoMessage() {} + +func (x *CreateServiceAccountRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[56] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateServiceAccountRequest.ProtoReflect.Descriptor instead. +func (*CreateServiceAccountRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{56} +} + +func (x *CreateServiceAccountRequest) GetSpec() *v1.ServiceAccountSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *CreateServiceAccountRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type CreateServiceAccountResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The ID of the created service account. + ServiceAccountId string `protobuf:"bytes,1,opt,name=service_account_id,json=serviceAccountId,proto3" json:"service_account_id,omitempty"` + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,2,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *CreateServiceAccountResponse) Reset() { + *x = CreateServiceAccountResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[57] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateServiceAccountResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateServiceAccountResponse) ProtoMessage() {} + +func (x *CreateServiceAccountResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[57] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateServiceAccountResponse.ProtoReflect.Descriptor instead. +func (*CreateServiceAccountResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{57} +} + +func (x *CreateServiceAccountResponse) GetServiceAccountId() string { + if x != nil { + return x.ServiceAccountId + } + return "" +} + +func (x *CreateServiceAccountResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type GetServiceAccountRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // ID of the service account to retrieve. + ServiceAccountId string `protobuf:"bytes,1,opt,name=service_account_id,json=serviceAccountId,proto3" json:"service_account_id,omitempty"` +} + +func (x *GetServiceAccountRequest) Reset() { + *x = GetServiceAccountRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[58] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetServiceAccountRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetServiceAccountRequest) ProtoMessage() {} + +func (x *GetServiceAccountRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[58] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetServiceAccountRequest.ProtoReflect.Descriptor instead. +func (*GetServiceAccountRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{58} +} + +func (x *GetServiceAccountRequest) GetServiceAccountId() string { + if x != nil { + return x.ServiceAccountId + } + return "" +} + +type GetServiceAccountResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The service account retrieved. + ServiceAccount *v1.ServiceAccount `protobuf:"bytes,1,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` +} + +func (x *GetServiceAccountResponse) Reset() { + *x = GetServiceAccountResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[59] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetServiceAccountResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetServiceAccountResponse) ProtoMessage() {} + +func (x *GetServiceAccountResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[59] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetServiceAccountResponse.ProtoReflect.Descriptor instead. +func (*GetServiceAccountResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{59} +} + +func (x *GetServiceAccountResponse) GetServiceAccount() *v1.ServiceAccount { + if x != nil { + return x.ServiceAccount + } + return nil +} + +type GetServiceAccountsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The requested size of the page to retrieve - optional. + // Cannot exceed 1000. Defaults to 100. + PageSize int32 `protobuf:"varint,1,opt,name=page_size,json=pageSize,proto3" json:"page_size,omitempty"` + // The page token if this is continuing from another response - optional. + PageToken string `protobuf:"bytes,2,opt,name=page_token,json=pageToken,proto3" json:"page_token,omitempty"` +} + +func (x *GetServiceAccountsRequest) Reset() { + *x = GetServiceAccountsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[60] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetServiceAccountsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetServiceAccountsRequest) ProtoMessage() {} + +func (x *GetServiceAccountsRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[60] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetServiceAccountsRequest.ProtoReflect.Descriptor instead. +func (*GetServiceAccountsRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{60} +} + +func (x *GetServiceAccountsRequest) GetPageSize() int32 { + if x != nil { + return x.PageSize + } + return 0 +} + +func (x *GetServiceAccountsRequest) GetPageToken() string { + if x != nil { + return x.PageToken + } + return "" +} + +type GetServiceAccountsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The list of service accounts in ascending ID order. + ServiceAccount []*v1.ServiceAccount `protobuf:"bytes,1,rep,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` + // The next page token, set if there is another page. + NextPageToken string `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *GetServiceAccountsResponse) Reset() { + *x = GetServiceAccountsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[61] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetServiceAccountsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetServiceAccountsResponse) ProtoMessage() {} + +func (x *GetServiceAccountsResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[61] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetServiceAccountsResponse.ProtoReflect.Descriptor instead. +func (*GetServiceAccountsResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{61} +} + +func (x *GetServiceAccountsResponse) GetServiceAccount() []*v1.ServiceAccount { + if x != nil { + return x.ServiceAccount + } + return nil +} + +func (x *GetServiceAccountsResponse) GetNextPageToken() string { + if x != nil { + return x.NextPageToken + } + return "" +} + +type UpdateServiceAccountRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The ID of the service account to update. + ServiceAccountId string `protobuf:"bytes,1,opt,name=service_account_id,json=serviceAccountId,proto3" json:"service_account_id,omitempty"` + // The new service account specification. + Spec *v1.ServiceAccountSpec `protobuf:"bytes,2,opt,name=spec,proto3" json:"spec,omitempty"` + // The version of the service account for which this update is intended for. + // The latest version can be found in the GetServiceAccount response. + ResourceVersion string `protobuf:"bytes,3,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The ID to use for this async operation - optional. + AsyncOperationId string `protobuf:"bytes,4,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *UpdateServiceAccountRequest) Reset() { + *x = UpdateServiceAccountRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[62] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateServiceAccountRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateServiceAccountRequest) ProtoMessage() {} + +func (x *UpdateServiceAccountRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[62] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateServiceAccountRequest.ProtoReflect.Descriptor instead. +func (*UpdateServiceAccountRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{62} +} + +func (x *UpdateServiceAccountRequest) GetServiceAccountId() string { + if x != nil { + return x.ServiceAccountId + } + return "" +} + +func (x *UpdateServiceAccountRequest) GetSpec() *v1.ServiceAccountSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *UpdateServiceAccountRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *UpdateServiceAccountRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type UpdateServiceAccountResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *UpdateServiceAccountResponse) Reset() { + *x = UpdateServiceAccountResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[63] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateServiceAccountResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateServiceAccountResponse) ProtoMessage() {} + +func (x *UpdateServiceAccountResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[63] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateServiceAccountResponse.ProtoReflect.Descriptor instead. +func (*UpdateServiceAccountResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{63} +} + +func (x *UpdateServiceAccountResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type DeleteServiceAccountRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The ID of the service account to delete; + ServiceAccountId string `protobuf:"bytes,1,opt,name=service_account_id,json=serviceAccountId,proto3" json:"service_account_id,omitempty"` + // The version of the service account for which this update is intended for. + // The latest version can be found in the GetServiceAccount response. + ResourceVersion string `protobuf:"bytes,2,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The ID to use for this async operation - optional. + AsyncOperationId string `protobuf:"bytes,3,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *DeleteServiceAccountRequest) Reset() { + *x = DeleteServiceAccountRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[64] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteServiceAccountRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteServiceAccountRequest) ProtoMessage() {} + +func (x *DeleteServiceAccountRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[64] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteServiceAccountRequest.ProtoReflect.Descriptor instead. +func (*DeleteServiceAccountRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{64} +} + +func (x *DeleteServiceAccountRequest) GetServiceAccountId() string { + if x != nil { + return x.ServiceAccountId + } + return "" +} + +func (x *DeleteServiceAccountRequest) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *DeleteServiceAccountRequest) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +type DeleteServiceAccountResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The async operation. + AsyncOperation *v11.AsyncOperation `protobuf:"bytes,1,opt,name=async_operation,json=asyncOperation,proto3" json:"async_operation,omitempty"` +} + +func (x *DeleteServiceAccountResponse) Reset() { + *x = DeleteServiceAccountResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[65] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteServiceAccountResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteServiceAccountResponse) ProtoMessage() {} + +func (x *DeleteServiceAccountResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[65] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteServiceAccountResponse.ProtoReflect.Descriptor instead. +func (*DeleteServiceAccountResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{65} +} + +func (x *DeleteServiceAccountResponse) GetAsyncOperation() *v11.AsyncOperation { + if x != nil { + return x.AsyncOperation + } + return nil +} + +type GetUserGroupsRequest_GoogleGroupFilter struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Filter groups by the google group email - optional. + EmailAddress string `protobuf:"bytes,1,opt,name=email_address,json=emailAddress,proto3" json:"email_address,omitempty"` +} + +func (x *GetUserGroupsRequest_GoogleGroupFilter) Reset() { + *x = GetUserGroupsRequest_GoogleGroupFilter{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[66] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetUserGroupsRequest_GoogleGroupFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetUserGroupsRequest_GoogleGroupFilter) ProtoMessage() {} + +func (x *GetUserGroupsRequest_GoogleGroupFilter) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[66] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetUserGroupsRequest_GoogleGroupFilter.ProtoReflect.Descriptor instead. +func (*GetUserGroupsRequest_GoogleGroupFilter) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP(), []int{44, 0} +} + +func (x *GetUserGroupsRequest_GoogleGroupFilter) GetEmailAddress() string { + if x != nil { + return x.EmailAddress + } + return "" +} + +var File_temporal_api_cloud_cloudservice_v1_request_response_proto protoreflect.FileDescriptor + +var file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDesc = []byte{ + 0x0a, 0x39, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, + 0x76, 0x31, 0x2f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x22, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x1a, 0x2d, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x6f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x2c, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, + 0x79, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x1a, 0x2d, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2f, 0x76, 0x31, + 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x2a, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, + 0x2f, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x91, 0x01, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x55, 0x73, + 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x09, 0x70, 0x61, 0x67, + 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x61, 0x67, + 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x70, 0x61, 0x67, 0x65, + 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x61, + 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x65, 0x6d, + 0x61, 0x69, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x6d, 0x61, 0x69, 0x6c, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x7e, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3e, 0x0a, 0x05, 0x75, 0x73, 0x65, 0x72, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, + 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x52, 0x05, 0x75, 0x73, 0x65, 0x72, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, + 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, + 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x2d, 0x0a, 0x0e, 0x47, + 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x07, + 0x75, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x75, 0x73, + 0x65, 0x72, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x4f, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x55, 0x73, + 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3c, 0x0a, 0x04, 0x75, 0x73, 0x65, + 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x52, 0x04, 0x75, 0x73, 0x65, 0x72, + 0x42, 0x02, 0x68, 0x00, 0x22, 0x87, 0x01, 0x0a, 0x11, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x55, 0x73, + 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x04, 0x73, 0x70, 0x65, + 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, + 0x73, 0x70, 0x65, 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, + 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8f, 0x01, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x55, 0x73, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1b, 0x0a, 0x07, + 0x75, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x75, 0x73, + 0x65, 0x72, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, + 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, + 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, + 0x22, 0xd3, 0x01, 0x0a, 0x11, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x07, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x75, 0x73, 0x65, 0x72, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x40, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, + 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, + 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x72, 0x0a, 0x12, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x91, 0x01, 0x0a, 0x11, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x07, + 0x75, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x75, 0x73, + 0x65, 0x72, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x22, 0x72, 0x0a, 0x12, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x55, 0x73, 0x65, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, + 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, + 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x8c, 0x02, 0x0a, 0x1d, 0x53, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x1b, 0x0a, 0x07, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x06, 0x75, 0x73, 0x65, 0x72, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x06, + 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, + 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x06, 0x61, 0x63, 0x63, 0x65, 0x73, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, + 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x7e, 0x0a, 0x1e, 0x53, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x4c, 0x0a, 0x18, 0x47, 0x65, 0x74, + 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, + 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x79, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, + 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x92, 0x01, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x46, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, + 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x22, 0x99, 0x01, 0x0a, 0x17, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x20, + 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, + 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x72, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, + 0x0a, 0x09, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x08, 0x70, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, + 0x0a, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x70, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x16, + 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x93, 0x01, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x4e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4e, 0x0a, + 0x0a, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x0a, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, + 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x37, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x64, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4c, + 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xe3, 0x01, 0x0a, 0x16, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x46, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, + 0x65, 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, + 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, + 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x77, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, 0x0a, 0x0f, 0x61, 0x73, + 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x22, 0xc9, 0x02, 0x0a, 0x22, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x43, 0x75, + 0x73, 0x74, 0x6f, 0x6d, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x54, 0x0a, 0x25, 0x65, 0x78, 0x69, 0x73, + 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x73, 0x65, 0x61, 0x72, 0x63, + 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x21, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x43, 0x75, + 0x73, 0x74, 0x6f, 0x6d, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x6e, 0x65, 0x77, + 0x5f, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x1c, 0x6e, 0x65, 0x77, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x53, 0x65, 0x61, 0x72, 0x63, + 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, + 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x83, 0x01, + 0x0a, 0x23, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x53, 0x65, 0x61, + 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x9b, 0x01, 0x0a, 0x16, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, + 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x22, 0x77, 0x0a, 0x17, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, 0x0a, 0x0f, 0x61, 0x73, + 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x90, 0x01, 0x0a, 0x1e, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x1a, 0x0a, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, + 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x7f, 0x0a, + 0x1f, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, + 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xba, 0x01, 0x0a, 0x19, 0x41, 0x64, 0x64, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x1a, 0x0a, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, + 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, + 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x7a, 0x0a, 0x1a, 0x41, 0x64, 0x64, + 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, + 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, + 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x13, + 0x0a, 0x11, 0x47, 0x65, 0x74, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x22, 0x58, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x07, 0x72, 0x65, 0x67, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x72, 0x65, 0x67, 0x69, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x72, 0x65, 0x67, + 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x2e, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x52, 0x65, + 0x67, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x06, 0x72, 0x65, + 0x67, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, + 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x55, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x52, 0x65, 0x67, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x40, 0x0a, 0x06, 0x72, 0x65, 0x67, 0x69, + 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x72, 0x65, 0x67, 0x69, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x72, 0x65, 0x67, + 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x99, 0x01, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x41, 0x70, + 0x69, 0x4b, 0x65, 0x79, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x09, 0x70, + 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, + 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x70, 0x61, + 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, + 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1d, 0x0a, 0x08, 0x6f, + 0x77, 0x6e, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6f, 0x77, + 0x6e, 0x65, 0x72, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x6f, 0x77, 0x6e, + 0x65, 0x72, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6f, 0x77, + 0x6e, 0x65, 0x72, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x87, 0x01, 0x0a, 0x12, 0x47, + 0x65, 0x74, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x45, 0x0a, 0x08, 0x61, 0x70, 0x69, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x26, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, + 0x2e, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x52, 0x07, 0x61, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, + 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, + 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x2d, 0x0a, 0x10, + 0x47, 0x65, 0x74, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x19, 0x0a, 0x06, 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x6b, 0x65, 0x79, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x58, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x41, + 0x70, 0x69, 0x4b, 0x65, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x43, 0x0a, 0x07, + 0x61, 0x70, 0x69, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, + 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x4b, + 0x65, 0x79, 0x52, 0x06, 0x61, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8b, 0x01, + 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x42, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, + 0x31, 0x2e, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, + 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x22, 0xa9, 0x01, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x41, 0x70, + 0x69, 0x4b, 0x65, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x19, 0x0a, 0x06, 0x6b, + 0x65, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6b, 0x65, 0x79, 0x49, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, + 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd5, 0x01, 0x0a, 0x13, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x19, 0x0a, 0x06, + 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6b, 0x65, 0x79, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x42, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, + 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, + 0x70, 0x65, 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, + 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x74, 0x0a, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x41, 0x70, 0x69, 0x4b, 0x65, + 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, + 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, + 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x91, 0x01, 0x0a, 0x13, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x41, 0x70, 0x69, 0x4b, 0x65, + 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x19, 0x0a, 0x06, 0x6b, 0x65, 0x79, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6b, 0x65, 0x79, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, + 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x74, 0x0a, + 0x14, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, + 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, + 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd4, 0x02, 0x0a, + 0x14, 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x09, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, + 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x71, 0x0a, 0x0c, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4a, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x47, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, + 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x0b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x47, 0x72, 0x6f, + 0x75, 0x70, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x3c, 0x0a, 0x11, 0x47, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x47, + 0x72, 0x6f, 0x75, 0x70, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0d, 0x65, 0x6d, 0x61, + 0x69, 0x6c, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0c, 0x65, 0x6d, 0x61, 0x69, 0x6c, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x22, 0x8a, 0x01, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x45, 0x0a, 0x06, 0x67, 0x72, 0x6f, 0x75, + 0x70, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, + 0x52, 0x06, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, + 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, + 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x34, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x47, + 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x08, 0x67, 0x72, + 0x6f, 0x75, 0x70, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x67, 0x72, 0x6f, + 0x75, 0x70, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x5b, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x55, + 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x43, 0x0a, 0x05, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, 0x65, + 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x05, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x02, 0x68, 0x00, + 0x22, 0x91, 0x01, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, + 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x45, 0x0a, 0x04, 0x73, 0x70, 0x65, + 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, + 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, + 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x96, 0x01, 0x0a, 0x17, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x08, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, + 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xdf, 0x01, 0x0a, + 0x16, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x08, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x45, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, + 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, + 0x65, 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, + 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, + 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x77, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, + 0x6f, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, 0x0a, 0x0f, 0x61, 0x73, + 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x22, 0x98, 0x01, 0x0a, 0x16, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x55, 0x73, + 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, + 0x08, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x67, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, + 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x77, 0x0a, 0x17, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x55, + 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x93, 0x02, 0x0a, 0x22, 0x53, 0x65, 0x74, 0x55, + 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1d, 0x0a, + 0x08, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x67, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x06, 0x61, 0x63, + 0x63, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x06, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x83, + 0x01, 0x0a, 0x23, 0x53, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, + 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, + 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, + 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x9b, 0x01, + 0x0a, 0x1b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, + 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4a, 0x0a, 0x04, 0x73, + 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, + 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, + 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x22, 0xae, 0x01, 0x0a, 0x1c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x4c, 0x0a, 0x18, 0x47, 0x65, 0x74, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, + 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x78, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5b, + 0x0a, 0x0f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x52, 0x0e, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0x5f, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x1f, 0x0a, 0x09, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x21, 0x0a, 0x0a, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x22, 0xa5, 0x01, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, + 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5b, + 0x0a, 0x0f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, + 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x0e, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, + 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, + 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, + 0x22, 0xfc, 0x01, 0x0a, 0x1b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, + 0x0a, 0x12, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, + 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, + 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, + 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x7c, 0x0a, 0x1c, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xb0, 0x01, 0x0a, 0x1b, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x5f, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x10, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x49, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, + 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x22, 0x7c, 0x0a, 0x1c, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x5c, 0x0a, 0x0f, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x42, 0xc8, 0x01, 0x0a, 0x25, 0x69, 0x6f, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, + 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x42, + 0x14, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x50, + 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x35, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, 0x76, 0x31, 0x3b, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0xaa, 0x02, 0x24, 0x54, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x2e, + 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x56, 0x31, 0xea, 0x02, + 0x28, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, + 0x3a, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x3a, 0x3a, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescOnce sync.Once + file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescData = file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDesc +) + +func file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescGZIP() []byte { + file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescOnce.Do(func() { + file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescData) + }) + return file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDescData +} + +var file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes = make([]protoimpl.MessageInfo, 67) +var file_temporal_api_cloud_cloudservice_v1_request_response_proto_goTypes = []any{ + (*GetUsersRequest)(nil), // 0: temporal.api.cloud.cloudservice.v1.GetUsersRequest + (*GetUsersResponse)(nil), // 1: temporal.api.cloud.cloudservice.v1.GetUsersResponse + (*GetUserRequest)(nil), // 2: temporal.api.cloud.cloudservice.v1.GetUserRequest + (*GetUserResponse)(nil), // 3: temporal.api.cloud.cloudservice.v1.GetUserResponse + (*CreateUserRequest)(nil), // 4: temporal.api.cloud.cloudservice.v1.CreateUserRequest + (*CreateUserResponse)(nil), // 5: temporal.api.cloud.cloudservice.v1.CreateUserResponse + (*UpdateUserRequest)(nil), // 6: temporal.api.cloud.cloudservice.v1.UpdateUserRequest + (*UpdateUserResponse)(nil), // 7: temporal.api.cloud.cloudservice.v1.UpdateUserResponse + (*DeleteUserRequest)(nil), // 8: temporal.api.cloud.cloudservice.v1.DeleteUserRequest + (*DeleteUserResponse)(nil), // 9: temporal.api.cloud.cloudservice.v1.DeleteUserResponse + (*SetUserNamespaceAccessRequest)(nil), // 10: temporal.api.cloud.cloudservice.v1.SetUserNamespaceAccessRequest + (*SetUserNamespaceAccessResponse)(nil), // 11: temporal.api.cloud.cloudservice.v1.SetUserNamespaceAccessResponse + (*GetAsyncOperationRequest)(nil), // 12: temporal.api.cloud.cloudservice.v1.GetAsyncOperationRequest + (*GetAsyncOperationResponse)(nil), // 13: temporal.api.cloud.cloudservice.v1.GetAsyncOperationResponse + (*CreateNamespaceRequest)(nil), // 14: temporal.api.cloud.cloudservice.v1.CreateNamespaceRequest + (*CreateNamespaceResponse)(nil), // 15: temporal.api.cloud.cloudservice.v1.CreateNamespaceResponse + (*GetNamespacesRequest)(nil), // 16: temporal.api.cloud.cloudservice.v1.GetNamespacesRequest + (*GetNamespacesResponse)(nil), // 17: temporal.api.cloud.cloudservice.v1.GetNamespacesResponse + (*GetNamespaceRequest)(nil), // 18: temporal.api.cloud.cloudservice.v1.GetNamespaceRequest + (*GetNamespaceResponse)(nil), // 19: temporal.api.cloud.cloudservice.v1.GetNamespaceResponse + (*UpdateNamespaceRequest)(nil), // 20: temporal.api.cloud.cloudservice.v1.UpdateNamespaceRequest + (*UpdateNamespaceResponse)(nil), // 21: temporal.api.cloud.cloudservice.v1.UpdateNamespaceResponse + (*RenameCustomSearchAttributeRequest)(nil), // 22: temporal.api.cloud.cloudservice.v1.RenameCustomSearchAttributeRequest + (*RenameCustomSearchAttributeResponse)(nil), // 23: temporal.api.cloud.cloudservice.v1.RenameCustomSearchAttributeResponse + (*DeleteNamespaceRequest)(nil), // 24: temporal.api.cloud.cloudservice.v1.DeleteNamespaceRequest + (*DeleteNamespaceResponse)(nil), // 25: temporal.api.cloud.cloudservice.v1.DeleteNamespaceResponse + (*FailoverNamespaceRegionRequest)(nil), // 26: temporal.api.cloud.cloudservice.v1.FailoverNamespaceRegionRequest + (*FailoverNamespaceRegionResponse)(nil), // 27: temporal.api.cloud.cloudservice.v1.FailoverNamespaceRegionResponse + (*AddNamespaceRegionRequest)(nil), // 28: temporal.api.cloud.cloudservice.v1.AddNamespaceRegionRequest + (*AddNamespaceRegionResponse)(nil), // 29: temporal.api.cloud.cloudservice.v1.AddNamespaceRegionResponse + (*GetRegionsRequest)(nil), // 30: temporal.api.cloud.cloudservice.v1.GetRegionsRequest + (*GetRegionsResponse)(nil), // 31: temporal.api.cloud.cloudservice.v1.GetRegionsResponse + (*GetRegionRequest)(nil), // 32: temporal.api.cloud.cloudservice.v1.GetRegionRequest + (*GetRegionResponse)(nil), // 33: temporal.api.cloud.cloudservice.v1.GetRegionResponse + (*GetApiKeysRequest)(nil), // 34: temporal.api.cloud.cloudservice.v1.GetApiKeysRequest + (*GetApiKeysResponse)(nil), // 35: temporal.api.cloud.cloudservice.v1.GetApiKeysResponse + (*GetApiKeyRequest)(nil), // 36: temporal.api.cloud.cloudservice.v1.GetApiKeyRequest + (*GetApiKeyResponse)(nil), // 37: temporal.api.cloud.cloudservice.v1.GetApiKeyResponse + (*CreateApiKeyRequest)(nil), // 38: temporal.api.cloud.cloudservice.v1.CreateApiKeyRequest + (*CreateApiKeyResponse)(nil), // 39: temporal.api.cloud.cloudservice.v1.CreateApiKeyResponse + (*UpdateApiKeyRequest)(nil), // 40: temporal.api.cloud.cloudservice.v1.UpdateApiKeyRequest + (*UpdateApiKeyResponse)(nil), // 41: temporal.api.cloud.cloudservice.v1.UpdateApiKeyResponse + (*DeleteApiKeyRequest)(nil), // 42: temporal.api.cloud.cloudservice.v1.DeleteApiKeyRequest + (*DeleteApiKeyResponse)(nil), // 43: temporal.api.cloud.cloudservice.v1.DeleteApiKeyResponse + (*GetUserGroupsRequest)(nil), // 44: temporal.api.cloud.cloudservice.v1.GetUserGroupsRequest + (*GetUserGroupsResponse)(nil), // 45: temporal.api.cloud.cloudservice.v1.GetUserGroupsResponse + (*GetUserGroupRequest)(nil), // 46: temporal.api.cloud.cloudservice.v1.GetUserGroupRequest + (*GetUserGroupResponse)(nil), // 47: temporal.api.cloud.cloudservice.v1.GetUserGroupResponse + (*CreateUserGroupRequest)(nil), // 48: temporal.api.cloud.cloudservice.v1.CreateUserGroupRequest + (*CreateUserGroupResponse)(nil), // 49: temporal.api.cloud.cloudservice.v1.CreateUserGroupResponse + (*UpdateUserGroupRequest)(nil), // 50: temporal.api.cloud.cloudservice.v1.UpdateUserGroupRequest + (*UpdateUserGroupResponse)(nil), // 51: temporal.api.cloud.cloudservice.v1.UpdateUserGroupResponse + (*DeleteUserGroupRequest)(nil), // 52: temporal.api.cloud.cloudservice.v1.DeleteUserGroupRequest + (*DeleteUserGroupResponse)(nil), // 53: temporal.api.cloud.cloudservice.v1.DeleteUserGroupResponse + (*SetUserGroupNamespaceAccessRequest)(nil), // 54: temporal.api.cloud.cloudservice.v1.SetUserGroupNamespaceAccessRequest + (*SetUserGroupNamespaceAccessResponse)(nil), // 55: temporal.api.cloud.cloudservice.v1.SetUserGroupNamespaceAccessResponse + (*CreateServiceAccountRequest)(nil), // 56: temporal.api.cloud.cloudservice.v1.CreateServiceAccountRequest + (*CreateServiceAccountResponse)(nil), // 57: temporal.api.cloud.cloudservice.v1.CreateServiceAccountResponse + (*GetServiceAccountRequest)(nil), // 58: temporal.api.cloud.cloudservice.v1.GetServiceAccountRequest + (*GetServiceAccountResponse)(nil), // 59: temporal.api.cloud.cloudservice.v1.GetServiceAccountResponse + (*GetServiceAccountsRequest)(nil), // 60: temporal.api.cloud.cloudservice.v1.GetServiceAccountsRequest + (*GetServiceAccountsResponse)(nil), // 61: temporal.api.cloud.cloudservice.v1.GetServiceAccountsResponse + (*UpdateServiceAccountRequest)(nil), // 62: temporal.api.cloud.cloudservice.v1.UpdateServiceAccountRequest + (*UpdateServiceAccountResponse)(nil), // 63: temporal.api.cloud.cloudservice.v1.UpdateServiceAccountResponse + (*DeleteServiceAccountRequest)(nil), // 64: temporal.api.cloud.cloudservice.v1.DeleteServiceAccountRequest + (*DeleteServiceAccountResponse)(nil), // 65: temporal.api.cloud.cloudservice.v1.DeleteServiceAccountResponse + (*GetUserGroupsRequest_GoogleGroupFilter)(nil), // 66: temporal.api.cloud.cloudservice.v1.GetUserGroupsRequest.GoogleGroupFilter + (*v1.User)(nil), // 67: temporal.api.cloud.identity.v1.User + (*v1.UserSpec)(nil), // 68: temporal.api.cloud.identity.v1.UserSpec + (*v11.AsyncOperation)(nil), // 69: temporal.api.cloud.operation.v1.AsyncOperation + (*v1.NamespaceAccess)(nil), // 70: temporal.api.cloud.identity.v1.NamespaceAccess + (*v12.NamespaceSpec)(nil), // 71: temporal.api.cloud.namespace.v1.NamespaceSpec + (*v12.Namespace)(nil), // 72: temporal.api.cloud.namespace.v1.Namespace + (*v13.Region)(nil), // 73: temporal.api.cloud.region.v1.Region + (*v1.ApiKey)(nil), // 74: temporal.api.cloud.identity.v1.ApiKey + (*v1.ApiKeySpec)(nil), // 75: temporal.api.cloud.identity.v1.ApiKeySpec + (*v1.UserGroup)(nil), // 76: temporal.api.cloud.identity.v1.UserGroup + (*v1.UserGroupSpec)(nil), // 77: temporal.api.cloud.identity.v1.UserGroupSpec + (*v1.ServiceAccountSpec)(nil), // 78: temporal.api.cloud.identity.v1.ServiceAccountSpec + (*v1.ServiceAccount)(nil), // 79: temporal.api.cloud.identity.v1.ServiceAccount +} +var file_temporal_api_cloud_cloudservice_v1_request_response_proto_depIdxs = []int32{ + 67, // 0: temporal.api.cloud.cloudservice.v1.GetUsersResponse.users:type_name -> temporal.api.cloud.identity.v1.User + 67, // 1: temporal.api.cloud.cloudservice.v1.GetUserResponse.user:type_name -> temporal.api.cloud.identity.v1.User + 68, // 2: temporal.api.cloud.cloudservice.v1.CreateUserRequest.spec:type_name -> temporal.api.cloud.identity.v1.UserSpec + 69, // 3: temporal.api.cloud.cloudservice.v1.CreateUserResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 68, // 4: temporal.api.cloud.cloudservice.v1.UpdateUserRequest.spec:type_name -> temporal.api.cloud.identity.v1.UserSpec + 69, // 5: temporal.api.cloud.cloudservice.v1.UpdateUserResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 69, // 6: temporal.api.cloud.cloudservice.v1.DeleteUserResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 70, // 7: temporal.api.cloud.cloudservice.v1.SetUserNamespaceAccessRequest.access:type_name -> temporal.api.cloud.identity.v1.NamespaceAccess + 69, // 8: temporal.api.cloud.cloudservice.v1.SetUserNamespaceAccessResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 69, // 9: temporal.api.cloud.cloudservice.v1.GetAsyncOperationResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 71, // 10: temporal.api.cloud.cloudservice.v1.CreateNamespaceRequest.spec:type_name -> temporal.api.cloud.namespace.v1.NamespaceSpec + 69, // 11: temporal.api.cloud.cloudservice.v1.CreateNamespaceResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 72, // 12: temporal.api.cloud.cloudservice.v1.GetNamespacesResponse.namespaces:type_name -> temporal.api.cloud.namespace.v1.Namespace + 72, // 13: temporal.api.cloud.cloudservice.v1.GetNamespaceResponse.namespace:type_name -> temporal.api.cloud.namespace.v1.Namespace + 71, // 14: temporal.api.cloud.cloudservice.v1.UpdateNamespaceRequest.spec:type_name -> temporal.api.cloud.namespace.v1.NamespaceSpec + 69, // 15: temporal.api.cloud.cloudservice.v1.UpdateNamespaceResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 69, // 16: temporal.api.cloud.cloudservice.v1.RenameCustomSearchAttributeResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 69, // 17: temporal.api.cloud.cloudservice.v1.DeleteNamespaceResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 69, // 18: temporal.api.cloud.cloudservice.v1.FailoverNamespaceRegionResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 69, // 19: temporal.api.cloud.cloudservice.v1.AddNamespaceRegionResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 73, // 20: temporal.api.cloud.cloudservice.v1.GetRegionsResponse.regions:type_name -> temporal.api.cloud.region.v1.Region + 73, // 21: temporal.api.cloud.cloudservice.v1.GetRegionResponse.region:type_name -> temporal.api.cloud.region.v1.Region + 74, // 22: temporal.api.cloud.cloudservice.v1.GetApiKeysResponse.api_keys:type_name -> temporal.api.cloud.identity.v1.ApiKey + 74, // 23: temporal.api.cloud.cloudservice.v1.GetApiKeyResponse.api_key:type_name -> temporal.api.cloud.identity.v1.ApiKey + 75, // 24: temporal.api.cloud.cloudservice.v1.CreateApiKeyRequest.spec:type_name -> temporal.api.cloud.identity.v1.ApiKeySpec + 69, // 25: temporal.api.cloud.cloudservice.v1.CreateApiKeyResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 75, // 26: temporal.api.cloud.cloudservice.v1.UpdateApiKeyRequest.spec:type_name -> temporal.api.cloud.identity.v1.ApiKeySpec + 69, // 27: temporal.api.cloud.cloudservice.v1.UpdateApiKeyResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 69, // 28: temporal.api.cloud.cloudservice.v1.DeleteApiKeyResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 66, // 29: temporal.api.cloud.cloudservice.v1.GetUserGroupsRequest.google_group:type_name -> temporal.api.cloud.cloudservice.v1.GetUserGroupsRequest.GoogleGroupFilter + 76, // 30: temporal.api.cloud.cloudservice.v1.GetUserGroupsResponse.groups:type_name -> temporal.api.cloud.identity.v1.UserGroup + 76, // 31: temporal.api.cloud.cloudservice.v1.GetUserGroupResponse.group:type_name -> temporal.api.cloud.identity.v1.UserGroup + 77, // 32: temporal.api.cloud.cloudservice.v1.CreateUserGroupRequest.spec:type_name -> temporal.api.cloud.identity.v1.UserGroupSpec + 69, // 33: temporal.api.cloud.cloudservice.v1.CreateUserGroupResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 77, // 34: temporal.api.cloud.cloudservice.v1.UpdateUserGroupRequest.spec:type_name -> temporal.api.cloud.identity.v1.UserGroupSpec + 69, // 35: temporal.api.cloud.cloudservice.v1.UpdateUserGroupResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 69, // 36: temporal.api.cloud.cloudservice.v1.DeleteUserGroupResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 70, // 37: temporal.api.cloud.cloudservice.v1.SetUserGroupNamespaceAccessRequest.access:type_name -> temporal.api.cloud.identity.v1.NamespaceAccess + 69, // 38: temporal.api.cloud.cloudservice.v1.SetUserGroupNamespaceAccessResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 78, // 39: temporal.api.cloud.cloudservice.v1.CreateServiceAccountRequest.spec:type_name -> temporal.api.cloud.identity.v1.ServiceAccountSpec + 69, // 40: temporal.api.cloud.cloudservice.v1.CreateServiceAccountResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 79, // 41: temporal.api.cloud.cloudservice.v1.GetServiceAccountResponse.service_account:type_name -> temporal.api.cloud.identity.v1.ServiceAccount + 79, // 42: temporal.api.cloud.cloudservice.v1.GetServiceAccountsResponse.service_account:type_name -> temporal.api.cloud.identity.v1.ServiceAccount + 78, // 43: temporal.api.cloud.cloudservice.v1.UpdateServiceAccountRequest.spec:type_name -> temporal.api.cloud.identity.v1.ServiceAccountSpec + 69, // 44: temporal.api.cloud.cloudservice.v1.UpdateServiceAccountResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 69, // 45: temporal.api.cloud.cloudservice.v1.DeleteServiceAccountResponse.async_operation:type_name -> temporal.api.cloud.operation.v1.AsyncOperation + 46, // [46:46] is the sub-list for method output_type + 46, // [46:46] is the sub-list for method input_type + 46, // [46:46] is the sub-list for extension type_name + 46, // [46:46] is the sub-list for extension extendee + 0, // [0:46] is the sub-list for field type_name +} + +func init() { file_temporal_api_cloud_cloudservice_v1_request_response_proto_init() } +func file_temporal_api_cloud_cloudservice_v1_request_response_proto_init() { + if File_temporal_api_cloud_cloudservice_v1_request_response_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*GetUsersRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*GetUsersResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*GetUserRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*GetUserResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*CreateUserRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*CreateUserResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*UpdateUserRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*UpdateUserResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*DeleteUserRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*DeleteUserResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*SetUserNamespaceAccessRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[11].Exporter = func(v any, i int) any { + switch v := v.(*SetUserNamespaceAccessResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[12].Exporter = func(v any, i int) any { + switch v := v.(*GetAsyncOperationRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[13].Exporter = func(v any, i int) any { + switch v := v.(*GetAsyncOperationResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[14].Exporter = func(v any, i int) any { + switch v := v.(*CreateNamespaceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[15].Exporter = func(v any, i int) any { + switch v := v.(*CreateNamespaceResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[16].Exporter = func(v any, i int) any { + switch v := v.(*GetNamespacesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[17].Exporter = func(v any, i int) any { + switch v := v.(*GetNamespacesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[18].Exporter = func(v any, i int) any { + switch v := v.(*GetNamespaceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[19].Exporter = func(v any, i int) any { + switch v := v.(*GetNamespaceResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[20].Exporter = func(v any, i int) any { + switch v := v.(*UpdateNamespaceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[21].Exporter = func(v any, i int) any { + switch v := v.(*UpdateNamespaceResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[22].Exporter = func(v any, i int) any { + switch v := v.(*RenameCustomSearchAttributeRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[23].Exporter = func(v any, i int) any { + switch v := v.(*RenameCustomSearchAttributeResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[24].Exporter = func(v any, i int) any { + switch v := v.(*DeleteNamespaceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[25].Exporter = func(v any, i int) any { + switch v := v.(*DeleteNamespaceResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[26].Exporter = func(v any, i int) any { + switch v := v.(*FailoverNamespaceRegionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[27].Exporter = func(v any, i int) any { + switch v := v.(*FailoverNamespaceRegionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[28].Exporter = func(v any, i int) any { + switch v := v.(*AddNamespaceRegionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[29].Exporter = func(v any, i int) any { + switch v := v.(*AddNamespaceRegionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[30].Exporter = func(v any, i int) any { + switch v := v.(*GetRegionsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[31].Exporter = func(v any, i int) any { + switch v := v.(*GetRegionsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[32].Exporter = func(v any, i int) any { + switch v := v.(*GetRegionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[33].Exporter = func(v any, i int) any { + switch v := v.(*GetRegionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[34].Exporter = func(v any, i int) any { + switch v := v.(*GetApiKeysRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[35].Exporter = func(v any, i int) any { + switch v := v.(*GetApiKeysResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[36].Exporter = func(v any, i int) any { + switch v := v.(*GetApiKeyRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[37].Exporter = func(v any, i int) any { + switch v := v.(*GetApiKeyResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[38].Exporter = func(v any, i int) any { + switch v := v.(*CreateApiKeyRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[39].Exporter = func(v any, i int) any { + switch v := v.(*CreateApiKeyResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[40].Exporter = func(v any, i int) any { + switch v := v.(*UpdateApiKeyRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[41].Exporter = func(v any, i int) any { + switch v := v.(*UpdateApiKeyResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[42].Exporter = func(v any, i int) any { + switch v := v.(*DeleteApiKeyRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[43].Exporter = func(v any, i int) any { + switch v := v.(*DeleteApiKeyResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[44].Exporter = func(v any, i int) any { + switch v := v.(*GetUserGroupsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[45].Exporter = func(v any, i int) any { + switch v := v.(*GetUserGroupsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[46].Exporter = func(v any, i int) any { + switch v := v.(*GetUserGroupRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[47].Exporter = func(v any, i int) any { + switch v := v.(*GetUserGroupResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[48].Exporter = func(v any, i int) any { + switch v := v.(*CreateUserGroupRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[49].Exporter = func(v any, i int) any { + switch v := v.(*CreateUserGroupResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[50].Exporter = func(v any, i int) any { + switch v := v.(*UpdateUserGroupRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[51].Exporter = func(v any, i int) any { + switch v := v.(*UpdateUserGroupResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[52].Exporter = func(v any, i int) any { + switch v := v.(*DeleteUserGroupRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[53].Exporter = func(v any, i int) any { + switch v := v.(*DeleteUserGroupResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[54].Exporter = func(v any, i int) any { + switch v := v.(*SetUserGroupNamespaceAccessRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[55].Exporter = func(v any, i int) any { + switch v := v.(*SetUserGroupNamespaceAccessResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[56].Exporter = func(v any, i int) any { + switch v := v.(*CreateServiceAccountRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[57].Exporter = func(v any, i int) any { + switch v := v.(*CreateServiceAccountResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[58].Exporter = func(v any, i int) any { + switch v := v.(*GetServiceAccountRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[59].Exporter = func(v any, i int) any { + switch v := v.(*GetServiceAccountResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[60].Exporter = func(v any, i int) any { + switch v := v.(*GetServiceAccountsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[61].Exporter = func(v any, i int) any { + switch v := v.(*GetServiceAccountsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[62].Exporter = func(v any, i int) any { + switch v := v.(*UpdateServiceAccountRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[63].Exporter = func(v any, i int) any { + switch v := v.(*UpdateServiceAccountResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[64].Exporter = func(v any, i int) any { + switch v := v.(*DeleteServiceAccountRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[65].Exporter = func(v any, i int) any { + switch v := v.(*DeleteServiceAccountResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes[66].Exporter = func(v any, i int) any { + switch v := v.(*GetUserGroupsRequest_GoogleGroupFilter); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDesc, + NumEnums: 0, + NumMessages: 67, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_cloud_cloudservice_v1_request_response_proto_goTypes, + DependencyIndexes: file_temporal_api_cloud_cloudservice_v1_request_response_proto_depIdxs, + MessageInfos: file_temporal_api_cloud_cloudservice_v1_request_response_proto_msgTypes, + }.Build() + File_temporal_api_cloud_cloudservice_v1_request_response_proto = out.File + file_temporal_api_cloud_cloudservice_v1_request_response_proto_rawDesc = nil + file_temporal_api_cloud_cloudservice_v1_request_response_proto_goTypes = nil + file_temporal_api_cloud_cloudservice_v1_request_response_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/cloud/cloudservice/v1/service.pb.go b/vendor/go.temporal.io/api/cloud/cloudservice/v1/service.pb.go new file mode 100644 index 00000000000..a4137a17e61 --- /dev/null +++ b/vendor/go.temporal.io/api/cloud/cloudservice/v1/service.pb.go @@ -0,0 +1,613 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/cloud/cloudservice/v1/service.proto + +package cloudservice + +import ( + reflect "reflect" + + _ "google.golang.org/genproto/googleapis/api/annotations" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +var File_temporal_api_cloud_cloudservice_v1_service_proto protoreflect.FileDescriptor + +var file_temporal_api_cloud_cloudservice_v1_service_proto_rawDesc = []byte{ + 0x0a, 0x30, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x12, 0x22, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x1a, 0x39, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x61, 0x6e, + 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x32, + 0xce, 0x2e, 0x0a, 0x0c, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x12, 0x8b, 0x01, 0x0a, 0x08, 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x73, 0x12, 0x33, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x34, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x14, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x0e, + 0x12, 0x0c, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x12, 0x92, + 0x01, 0x0a, 0x07, 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x12, 0x32, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, + 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x33, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x1e, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x18, 0x12, 0x16, 0x2f, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x2f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x2f, 0x7b, 0x75, 0x73, 0x65, 0x72, 0x5f, + 0x69, 0x64, 0x7d, 0x12, 0x94, 0x01, 0x0a, 0x0a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x55, 0x73, + 0x65, 0x72, 0x12, 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x55, 0x73, + 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x17, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x11, 0x3a, 0x01, 0x2a, 0x22, 0x0c, 0x2f, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x12, 0x9e, 0x01, 0x0a, 0x0a, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x12, 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x36, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x21, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1b, + 0x3a, 0x01, 0x2a, 0x22, 0x16, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x75, 0x73, 0x65, 0x72, + 0x73, 0x2f, 0x7b, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0x9b, 0x01, 0x0a, 0x0a, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x12, 0x35, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, + 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x36, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x55, 0x73, 0x65, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1e, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0x18, 0x2a, 0x16, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x2f, + 0x7b, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0xe0, 0x01, 0x0a, 0x16, 0x53, 0x65, + 0x74, 0x55, 0x73, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x63, + 0x63, 0x65, 0x73, 0x73, 0x12, 0x41, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, 0x55, 0x73, 0x65, + 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x42, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, + 0x55, 0x73, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x63, 0x63, + 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3f, 0x82, 0xd3, 0xe4, + 0x93, 0x02, 0x39, 0x3a, 0x01, 0x2a, 0x22, 0x34, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x75, 0x73, 0x65, 0x72, 0x73, 0x2f, 0x7b, 0x75, 0x73, 0x65, + 0x72, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0xc0, 0x01, 0x0a, + 0x11, 0x47, 0x65, 0x74, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x3c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x73, 0x79, 0x6e, 0x63, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x3d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x2e, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x28, 0x12, 0x26, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, + 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x7b, 0x61, 0x73, 0x79, 0x6e, + 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x7d, 0x12, + 0xa8, 0x01, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x12, 0x3a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1c, 0x82, 0xd3, + 0xe4, 0x93, 0x02, 0x16, 0x3a, 0x01, 0x2a, 0x22, 0x11, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x12, 0x9f, 0x01, 0x0a, 0x0d, 0x47, + 0x65, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x12, 0x38, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x39, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x19, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x13, 0x12, 0x11, 0x2f, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x12, 0xa8, 0x01, 0x0a, + 0x0c, 0x47, 0x65, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x37, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x38, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x25, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1f, 0x12, 0x1d, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, + 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x12, 0xb4, 0x01, 0x0a, 0x0f, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x3a, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, + 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x28, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x22, 0x3a, 0x01, 0x2a, 0x22, + 0x1d, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x12, 0xf7, + 0x01, 0x0a, 0x1b, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x53, + 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x12, 0x46, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, + 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x47, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6e, 0x61, + 0x6d, 0x65, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x47, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x41, 0x3a, 0x01, 0x2a, 0x22, 0x3c, 0x2f, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, + 0x2d, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x2d, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x2d, 0x61, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x12, 0xb1, 0x01, 0x0a, 0x0f, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x3a, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x25, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1f, 0x2a, 0x1d, 0x2f, + 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, + 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x12, 0xdc, 0x01, 0x0a, + 0x17, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x12, 0x42, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, + 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x43, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x38, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x32, 0x3a, 0x01, 0x2a, 0x22, 0x2d, 0x2f, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, + 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x66, 0x61, 0x69, 0x6c, + 0x6f, 0x76, 0x65, 0x72, 0x2d, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x12, 0xc8, 0x01, 0x0a, 0x12, + 0x41, 0x64, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x67, 0x69, + 0x6f, 0x6e, 0x12, 0x3d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x64, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x64, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x33, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2d, 0x3a, 0x01, 0x2a, 0x22, 0x28, 0x2f, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, + 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x64, 0x64, 0x2d, + 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x12, 0x93, 0x01, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x52, 0x65, + 0x67, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, + 0x67, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x16, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x10, 0x12, 0x0e, 0x2f, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x99, 0x01, 0x0a, + 0x09, 0x47, 0x65, 0x74, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x12, 0x34, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, + 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x47, 0x65, 0x74, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1f, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x19, 0x12, + 0x17, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x73, 0x2f, + 0x7b, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x7d, 0x12, 0x94, 0x01, 0x0a, 0x0a, 0x47, 0x65, 0x74, + 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, + 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x17, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x11, 0x12, 0x0f, + 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x61, 0x70, 0x69, 0x2d, 0x6b, 0x65, 0x79, 0x73, 0x12, + 0x9a, 0x01, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x12, 0x34, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x70, 0x69, 0x4b, + 0x65, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x20, 0x82, 0xd3, 0xe4, 0x93, + 0x02, 0x1a, 0x12, 0x18, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x61, 0x70, 0x69, 0x2d, 0x6b, + 0x65, 0x79, 0x73, 0x2f, 0x7b, 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0x9d, 0x01, 0x0a, + 0x0c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x12, 0x37, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x38, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x1a, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x14, 0x3a, 0x01, 0x2a, 0x22, 0x0f, 0x2f, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x2f, 0x61, 0x70, 0x69, 0x2d, 0x6b, 0x65, 0x79, 0x73, 0x12, 0xa6, 0x01, 0x0a, + 0x0c, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x12, 0x37, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x38, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x23, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1d, 0x3a, 0x01, 0x2a, 0x22, 0x18, 0x2f, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x2f, 0x61, 0x70, 0x69, 0x2d, 0x6b, 0x65, 0x79, 0x73, 0x2f, 0x7b, 0x6b, 0x65, + 0x79, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0xa3, 0x01, 0x0a, 0x0c, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x12, 0x37, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x38, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x41, 0x70, 0x69, 0x4b, 0x65, + 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x20, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0x1a, 0x2a, 0x18, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x61, 0x70, 0x69, 0x2d, 0x6b, 0x65, + 0x79, 0x73, 0x2f, 0x7b, 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0xa0, 0x01, 0x0a, 0x0d, + 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x12, 0x38, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x39, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, + 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x1a, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x14, 0x12, 0x12, 0x2f, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2f, 0x75, 0x73, 0x65, 0x72, 0x2d, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x12, 0xa8, + 0x01, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, + 0x37, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x38, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, + 0x74, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x25, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1f, 0x12, 0x1d, 0x2f, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2f, 0x75, 0x73, 0x65, 0x72, 0x2d, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x2f, 0x7b, + 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0xa9, 0x01, 0x0a, 0x0f, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x3a, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, + 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x17, 0x3a, 0x01, + 0x2a, 0x22, 0x12, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x75, 0x73, 0x65, 0x72, 0x2d, 0x67, + 0x72, 0x6f, 0x75, 0x70, 0x73, 0x12, 0xb4, 0x01, 0x0a, 0x0f, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x3a, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x28, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x22, 0x3a, 0x01, 0x2a, 0x22, 0x1d, 0x2f, + 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x75, 0x73, 0x65, 0x72, 0x2d, 0x67, 0x72, 0x6f, 0x75, 0x70, + 0x73, 0x2f, 0x7b, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0xb1, 0x01, 0x0a, + 0x0f, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, + 0x12, 0x3a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, + 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3b, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x25, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0x1f, 0x2a, 0x1d, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x75, 0x73, 0x65, 0x72, 0x2d, 0x67, + 0x72, 0x6f, 0x75, 0x70, 0x73, 0x2f, 0x7b, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x69, 0x64, 0x7d, + 0x12, 0xf6, 0x01, 0x0a, 0x1b, 0x53, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, + 0x12, 0x46, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, + 0x75, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x63, 0x63, 0x65, 0x73, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x47, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, + 0x74, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x46, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x40, 0x3a, 0x01, 0x2a, 0x22, 0x3b, 0x2f, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, + 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x75, 0x73, 0x65, 0x72, + 0x2d, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x2f, 0x7b, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x69, + 0x64, 0x7d, 0x2f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0xbd, 0x01, 0x0a, 0x14, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x12, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x22, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1c, 0x3a, 0x01, 0x2a, + 0x22, 0x17, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2d, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x12, 0xc6, 0x01, 0x0a, 0x11, 0x47, 0x65, + 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, + 0x3c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, + 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3d, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, + 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x34, 0x82, 0xd3, + 0xe4, 0x93, 0x02, 0x2e, 0x12, 0x2c, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x2d, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2f, 0x7b, 0x73, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x69, + 0x64, 0x7d, 0x12, 0xb4, 0x01, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x12, 0x3d, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, + 0x65, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, + 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1f, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x19, + 0x12, 0x17, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2d, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x12, 0xd2, 0x01, 0x0a, 0x14, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x12, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x37, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x31, 0x3a, 0x01, 0x2a, + 0x22, 0x2c, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2d, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2f, 0x7b, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0xcf, + 0x01, 0x0a, 0x14, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x34, 0x82, 0xd3, 0xe4, 0x93, + 0x02, 0x2e, 0x2a, 0x2c, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2d, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2f, 0x7b, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x7d, + 0x42, 0xc0, 0x01, 0x0a, 0x25, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x35, 0x67, 0x6f, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2f, 0x76, 0x31, 0x3b, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0xaa, 0x02, 0x24, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, + 0x70, 0x69, 0x2e, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x28, 0x54, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x43, 0x6c, 0x6f, 0x75, + 0x64, 0x3a, 0x3a, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x3a, + 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var file_temporal_api_cloud_cloudservice_v1_service_proto_goTypes = []any{ + (*GetUsersRequest)(nil), // 0: temporal.api.cloud.cloudservice.v1.GetUsersRequest + (*GetUserRequest)(nil), // 1: temporal.api.cloud.cloudservice.v1.GetUserRequest + (*CreateUserRequest)(nil), // 2: temporal.api.cloud.cloudservice.v1.CreateUserRequest + (*UpdateUserRequest)(nil), // 3: temporal.api.cloud.cloudservice.v1.UpdateUserRequest + (*DeleteUserRequest)(nil), // 4: temporal.api.cloud.cloudservice.v1.DeleteUserRequest + (*SetUserNamespaceAccessRequest)(nil), // 5: temporal.api.cloud.cloudservice.v1.SetUserNamespaceAccessRequest + (*GetAsyncOperationRequest)(nil), // 6: temporal.api.cloud.cloudservice.v1.GetAsyncOperationRequest + (*CreateNamespaceRequest)(nil), // 7: temporal.api.cloud.cloudservice.v1.CreateNamespaceRequest + (*GetNamespacesRequest)(nil), // 8: temporal.api.cloud.cloudservice.v1.GetNamespacesRequest + (*GetNamespaceRequest)(nil), // 9: temporal.api.cloud.cloudservice.v1.GetNamespaceRequest + (*UpdateNamespaceRequest)(nil), // 10: temporal.api.cloud.cloudservice.v1.UpdateNamespaceRequest + (*RenameCustomSearchAttributeRequest)(nil), // 11: temporal.api.cloud.cloudservice.v1.RenameCustomSearchAttributeRequest + (*DeleteNamespaceRequest)(nil), // 12: temporal.api.cloud.cloudservice.v1.DeleteNamespaceRequest + (*FailoverNamespaceRegionRequest)(nil), // 13: temporal.api.cloud.cloudservice.v1.FailoverNamespaceRegionRequest + (*AddNamespaceRegionRequest)(nil), // 14: temporal.api.cloud.cloudservice.v1.AddNamespaceRegionRequest + (*GetRegionsRequest)(nil), // 15: temporal.api.cloud.cloudservice.v1.GetRegionsRequest + (*GetRegionRequest)(nil), // 16: temporal.api.cloud.cloudservice.v1.GetRegionRequest + (*GetApiKeysRequest)(nil), // 17: temporal.api.cloud.cloudservice.v1.GetApiKeysRequest + (*GetApiKeyRequest)(nil), // 18: temporal.api.cloud.cloudservice.v1.GetApiKeyRequest + (*CreateApiKeyRequest)(nil), // 19: temporal.api.cloud.cloudservice.v1.CreateApiKeyRequest + (*UpdateApiKeyRequest)(nil), // 20: temporal.api.cloud.cloudservice.v1.UpdateApiKeyRequest + (*DeleteApiKeyRequest)(nil), // 21: temporal.api.cloud.cloudservice.v1.DeleteApiKeyRequest + (*GetUserGroupsRequest)(nil), // 22: temporal.api.cloud.cloudservice.v1.GetUserGroupsRequest + (*GetUserGroupRequest)(nil), // 23: temporal.api.cloud.cloudservice.v1.GetUserGroupRequest + (*CreateUserGroupRequest)(nil), // 24: temporal.api.cloud.cloudservice.v1.CreateUserGroupRequest + (*UpdateUserGroupRequest)(nil), // 25: temporal.api.cloud.cloudservice.v1.UpdateUserGroupRequest + (*DeleteUserGroupRequest)(nil), // 26: temporal.api.cloud.cloudservice.v1.DeleteUserGroupRequest + (*SetUserGroupNamespaceAccessRequest)(nil), // 27: temporal.api.cloud.cloudservice.v1.SetUserGroupNamespaceAccessRequest + (*CreateServiceAccountRequest)(nil), // 28: temporal.api.cloud.cloudservice.v1.CreateServiceAccountRequest + (*GetServiceAccountRequest)(nil), // 29: temporal.api.cloud.cloudservice.v1.GetServiceAccountRequest + (*GetServiceAccountsRequest)(nil), // 30: temporal.api.cloud.cloudservice.v1.GetServiceAccountsRequest + (*UpdateServiceAccountRequest)(nil), // 31: temporal.api.cloud.cloudservice.v1.UpdateServiceAccountRequest + (*DeleteServiceAccountRequest)(nil), // 32: temporal.api.cloud.cloudservice.v1.DeleteServiceAccountRequest + (*GetUsersResponse)(nil), // 33: temporal.api.cloud.cloudservice.v1.GetUsersResponse + (*GetUserResponse)(nil), // 34: temporal.api.cloud.cloudservice.v1.GetUserResponse + (*CreateUserResponse)(nil), // 35: temporal.api.cloud.cloudservice.v1.CreateUserResponse + (*UpdateUserResponse)(nil), // 36: temporal.api.cloud.cloudservice.v1.UpdateUserResponse + (*DeleteUserResponse)(nil), // 37: temporal.api.cloud.cloudservice.v1.DeleteUserResponse + (*SetUserNamespaceAccessResponse)(nil), // 38: temporal.api.cloud.cloudservice.v1.SetUserNamespaceAccessResponse + (*GetAsyncOperationResponse)(nil), // 39: temporal.api.cloud.cloudservice.v1.GetAsyncOperationResponse + (*CreateNamespaceResponse)(nil), // 40: temporal.api.cloud.cloudservice.v1.CreateNamespaceResponse + (*GetNamespacesResponse)(nil), // 41: temporal.api.cloud.cloudservice.v1.GetNamespacesResponse + (*GetNamespaceResponse)(nil), // 42: temporal.api.cloud.cloudservice.v1.GetNamespaceResponse + (*UpdateNamespaceResponse)(nil), // 43: temporal.api.cloud.cloudservice.v1.UpdateNamespaceResponse + (*RenameCustomSearchAttributeResponse)(nil), // 44: temporal.api.cloud.cloudservice.v1.RenameCustomSearchAttributeResponse + (*DeleteNamespaceResponse)(nil), // 45: temporal.api.cloud.cloudservice.v1.DeleteNamespaceResponse + (*FailoverNamespaceRegionResponse)(nil), // 46: temporal.api.cloud.cloudservice.v1.FailoverNamespaceRegionResponse + (*AddNamespaceRegionResponse)(nil), // 47: temporal.api.cloud.cloudservice.v1.AddNamespaceRegionResponse + (*GetRegionsResponse)(nil), // 48: temporal.api.cloud.cloudservice.v1.GetRegionsResponse + (*GetRegionResponse)(nil), // 49: temporal.api.cloud.cloudservice.v1.GetRegionResponse + (*GetApiKeysResponse)(nil), // 50: temporal.api.cloud.cloudservice.v1.GetApiKeysResponse + (*GetApiKeyResponse)(nil), // 51: temporal.api.cloud.cloudservice.v1.GetApiKeyResponse + (*CreateApiKeyResponse)(nil), // 52: temporal.api.cloud.cloudservice.v1.CreateApiKeyResponse + (*UpdateApiKeyResponse)(nil), // 53: temporal.api.cloud.cloudservice.v1.UpdateApiKeyResponse + (*DeleteApiKeyResponse)(nil), // 54: temporal.api.cloud.cloudservice.v1.DeleteApiKeyResponse + (*GetUserGroupsResponse)(nil), // 55: temporal.api.cloud.cloudservice.v1.GetUserGroupsResponse + (*GetUserGroupResponse)(nil), // 56: temporal.api.cloud.cloudservice.v1.GetUserGroupResponse + (*CreateUserGroupResponse)(nil), // 57: temporal.api.cloud.cloudservice.v1.CreateUserGroupResponse + (*UpdateUserGroupResponse)(nil), // 58: temporal.api.cloud.cloudservice.v1.UpdateUserGroupResponse + (*DeleteUserGroupResponse)(nil), // 59: temporal.api.cloud.cloudservice.v1.DeleteUserGroupResponse + (*SetUserGroupNamespaceAccessResponse)(nil), // 60: temporal.api.cloud.cloudservice.v1.SetUserGroupNamespaceAccessResponse + (*CreateServiceAccountResponse)(nil), // 61: temporal.api.cloud.cloudservice.v1.CreateServiceAccountResponse + (*GetServiceAccountResponse)(nil), // 62: temporal.api.cloud.cloudservice.v1.GetServiceAccountResponse + (*GetServiceAccountsResponse)(nil), // 63: temporal.api.cloud.cloudservice.v1.GetServiceAccountsResponse + (*UpdateServiceAccountResponse)(nil), // 64: temporal.api.cloud.cloudservice.v1.UpdateServiceAccountResponse + (*DeleteServiceAccountResponse)(nil), // 65: temporal.api.cloud.cloudservice.v1.DeleteServiceAccountResponse +} +var file_temporal_api_cloud_cloudservice_v1_service_proto_depIdxs = []int32{ + 0, // 0: temporal.api.cloud.cloudservice.v1.CloudService.GetUsers:input_type -> temporal.api.cloud.cloudservice.v1.GetUsersRequest + 1, // 1: temporal.api.cloud.cloudservice.v1.CloudService.GetUser:input_type -> temporal.api.cloud.cloudservice.v1.GetUserRequest + 2, // 2: temporal.api.cloud.cloudservice.v1.CloudService.CreateUser:input_type -> temporal.api.cloud.cloudservice.v1.CreateUserRequest + 3, // 3: temporal.api.cloud.cloudservice.v1.CloudService.UpdateUser:input_type -> temporal.api.cloud.cloudservice.v1.UpdateUserRequest + 4, // 4: temporal.api.cloud.cloudservice.v1.CloudService.DeleteUser:input_type -> temporal.api.cloud.cloudservice.v1.DeleteUserRequest + 5, // 5: temporal.api.cloud.cloudservice.v1.CloudService.SetUserNamespaceAccess:input_type -> temporal.api.cloud.cloudservice.v1.SetUserNamespaceAccessRequest + 6, // 6: temporal.api.cloud.cloudservice.v1.CloudService.GetAsyncOperation:input_type -> temporal.api.cloud.cloudservice.v1.GetAsyncOperationRequest + 7, // 7: temporal.api.cloud.cloudservice.v1.CloudService.CreateNamespace:input_type -> temporal.api.cloud.cloudservice.v1.CreateNamespaceRequest + 8, // 8: temporal.api.cloud.cloudservice.v1.CloudService.GetNamespaces:input_type -> temporal.api.cloud.cloudservice.v1.GetNamespacesRequest + 9, // 9: temporal.api.cloud.cloudservice.v1.CloudService.GetNamespace:input_type -> temporal.api.cloud.cloudservice.v1.GetNamespaceRequest + 10, // 10: temporal.api.cloud.cloudservice.v1.CloudService.UpdateNamespace:input_type -> temporal.api.cloud.cloudservice.v1.UpdateNamespaceRequest + 11, // 11: temporal.api.cloud.cloudservice.v1.CloudService.RenameCustomSearchAttribute:input_type -> temporal.api.cloud.cloudservice.v1.RenameCustomSearchAttributeRequest + 12, // 12: temporal.api.cloud.cloudservice.v1.CloudService.DeleteNamespace:input_type -> temporal.api.cloud.cloudservice.v1.DeleteNamespaceRequest + 13, // 13: temporal.api.cloud.cloudservice.v1.CloudService.FailoverNamespaceRegion:input_type -> temporal.api.cloud.cloudservice.v1.FailoverNamespaceRegionRequest + 14, // 14: temporal.api.cloud.cloudservice.v1.CloudService.AddNamespaceRegion:input_type -> temporal.api.cloud.cloudservice.v1.AddNamespaceRegionRequest + 15, // 15: temporal.api.cloud.cloudservice.v1.CloudService.GetRegions:input_type -> temporal.api.cloud.cloudservice.v1.GetRegionsRequest + 16, // 16: temporal.api.cloud.cloudservice.v1.CloudService.GetRegion:input_type -> temporal.api.cloud.cloudservice.v1.GetRegionRequest + 17, // 17: temporal.api.cloud.cloudservice.v1.CloudService.GetApiKeys:input_type -> temporal.api.cloud.cloudservice.v1.GetApiKeysRequest + 18, // 18: temporal.api.cloud.cloudservice.v1.CloudService.GetApiKey:input_type -> temporal.api.cloud.cloudservice.v1.GetApiKeyRequest + 19, // 19: temporal.api.cloud.cloudservice.v1.CloudService.CreateApiKey:input_type -> temporal.api.cloud.cloudservice.v1.CreateApiKeyRequest + 20, // 20: temporal.api.cloud.cloudservice.v1.CloudService.UpdateApiKey:input_type -> temporal.api.cloud.cloudservice.v1.UpdateApiKeyRequest + 21, // 21: temporal.api.cloud.cloudservice.v1.CloudService.DeleteApiKey:input_type -> temporal.api.cloud.cloudservice.v1.DeleteApiKeyRequest + 22, // 22: temporal.api.cloud.cloudservice.v1.CloudService.GetUserGroups:input_type -> temporal.api.cloud.cloudservice.v1.GetUserGroupsRequest + 23, // 23: temporal.api.cloud.cloudservice.v1.CloudService.GetUserGroup:input_type -> temporal.api.cloud.cloudservice.v1.GetUserGroupRequest + 24, // 24: temporal.api.cloud.cloudservice.v1.CloudService.CreateUserGroup:input_type -> temporal.api.cloud.cloudservice.v1.CreateUserGroupRequest + 25, // 25: temporal.api.cloud.cloudservice.v1.CloudService.UpdateUserGroup:input_type -> temporal.api.cloud.cloudservice.v1.UpdateUserGroupRequest + 26, // 26: temporal.api.cloud.cloudservice.v1.CloudService.DeleteUserGroup:input_type -> temporal.api.cloud.cloudservice.v1.DeleteUserGroupRequest + 27, // 27: temporal.api.cloud.cloudservice.v1.CloudService.SetUserGroupNamespaceAccess:input_type -> temporal.api.cloud.cloudservice.v1.SetUserGroupNamespaceAccessRequest + 28, // 28: temporal.api.cloud.cloudservice.v1.CloudService.CreateServiceAccount:input_type -> temporal.api.cloud.cloudservice.v1.CreateServiceAccountRequest + 29, // 29: temporal.api.cloud.cloudservice.v1.CloudService.GetServiceAccount:input_type -> temporal.api.cloud.cloudservice.v1.GetServiceAccountRequest + 30, // 30: temporal.api.cloud.cloudservice.v1.CloudService.GetServiceAccounts:input_type -> temporal.api.cloud.cloudservice.v1.GetServiceAccountsRequest + 31, // 31: temporal.api.cloud.cloudservice.v1.CloudService.UpdateServiceAccount:input_type -> temporal.api.cloud.cloudservice.v1.UpdateServiceAccountRequest + 32, // 32: temporal.api.cloud.cloudservice.v1.CloudService.DeleteServiceAccount:input_type -> temporal.api.cloud.cloudservice.v1.DeleteServiceAccountRequest + 33, // 33: temporal.api.cloud.cloudservice.v1.CloudService.GetUsers:output_type -> temporal.api.cloud.cloudservice.v1.GetUsersResponse + 34, // 34: temporal.api.cloud.cloudservice.v1.CloudService.GetUser:output_type -> temporal.api.cloud.cloudservice.v1.GetUserResponse + 35, // 35: temporal.api.cloud.cloudservice.v1.CloudService.CreateUser:output_type -> temporal.api.cloud.cloudservice.v1.CreateUserResponse + 36, // 36: temporal.api.cloud.cloudservice.v1.CloudService.UpdateUser:output_type -> temporal.api.cloud.cloudservice.v1.UpdateUserResponse + 37, // 37: temporal.api.cloud.cloudservice.v1.CloudService.DeleteUser:output_type -> temporal.api.cloud.cloudservice.v1.DeleteUserResponse + 38, // 38: temporal.api.cloud.cloudservice.v1.CloudService.SetUserNamespaceAccess:output_type -> temporal.api.cloud.cloudservice.v1.SetUserNamespaceAccessResponse + 39, // 39: temporal.api.cloud.cloudservice.v1.CloudService.GetAsyncOperation:output_type -> temporal.api.cloud.cloudservice.v1.GetAsyncOperationResponse + 40, // 40: temporal.api.cloud.cloudservice.v1.CloudService.CreateNamespace:output_type -> temporal.api.cloud.cloudservice.v1.CreateNamespaceResponse + 41, // 41: temporal.api.cloud.cloudservice.v1.CloudService.GetNamespaces:output_type -> temporal.api.cloud.cloudservice.v1.GetNamespacesResponse + 42, // 42: temporal.api.cloud.cloudservice.v1.CloudService.GetNamespace:output_type -> temporal.api.cloud.cloudservice.v1.GetNamespaceResponse + 43, // 43: temporal.api.cloud.cloudservice.v1.CloudService.UpdateNamespace:output_type -> temporal.api.cloud.cloudservice.v1.UpdateNamespaceResponse + 44, // 44: temporal.api.cloud.cloudservice.v1.CloudService.RenameCustomSearchAttribute:output_type -> temporal.api.cloud.cloudservice.v1.RenameCustomSearchAttributeResponse + 45, // 45: temporal.api.cloud.cloudservice.v1.CloudService.DeleteNamespace:output_type -> temporal.api.cloud.cloudservice.v1.DeleteNamespaceResponse + 46, // 46: temporal.api.cloud.cloudservice.v1.CloudService.FailoverNamespaceRegion:output_type -> temporal.api.cloud.cloudservice.v1.FailoverNamespaceRegionResponse + 47, // 47: temporal.api.cloud.cloudservice.v1.CloudService.AddNamespaceRegion:output_type -> temporal.api.cloud.cloudservice.v1.AddNamespaceRegionResponse + 48, // 48: temporal.api.cloud.cloudservice.v1.CloudService.GetRegions:output_type -> temporal.api.cloud.cloudservice.v1.GetRegionsResponse + 49, // 49: temporal.api.cloud.cloudservice.v1.CloudService.GetRegion:output_type -> temporal.api.cloud.cloudservice.v1.GetRegionResponse + 50, // 50: temporal.api.cloud.cloudservice.v1.CloudService.GetApiKeys:output_type -> temporal.api.cloud.cloudservice.v1.GetApiKeysResponse + 51, // 51: temporal.api.cloud.cloudservice.v1.CloudService.GetApiKey:output_type -> temporal.api.cloud.cloudservice.v1.GetApiKeyResponse + 52, // 52: temporal.api.cloud.cloudservice.v1.CloudService.CreateApiKey:output_type -> temporal.api.cloud.cloudservice.v1.CreateApiKeyResponse + 53, // 53: temporal.api.cloud.cloudservice.v1.CloudService.UpdateApiKey:output_type -> temporal.api.cloud.cloudservice.v1.UpdateApiKeyResponse + 54, // 54: temporal.api.cloud.cloudservice.v1.CloudService.DeleteApiKey:output_type -> temporal.api.cloud.cloudservice.v1.DeleteApiKeyResponse + 55, // 55: temporal.api.cloud.cloudservice.v1.CloudService.GetUserGroups:output_type -> temporal.api.cloud.cloudservice.v1.GetUserGroupsResponse + 56, // 56: temporal.api.cloud.cloudservice.v1.CloudService.GetUserGroup:output_type -> temporal.api.cloud.cloudservice.v1.GetUserGroupResponse + 57, // 57: temporal.api.cloud.cloudservice.v1.CloudService.CreateUserGroup:output_type -> temporal.api.cloud.cloudservice.v1.CreateUserGroupResponse + 58, // 58: temporal.api.cloud.cloudservice.v1.CloudService.UpdateUserGroup:output_type -> temporal.api.cloud.cloudservice.v1.UpdateUserGroupResponse + 59, // 59: temporal.api.cloud.cloudservice.v1.CloudService.DeleteUserGroup:output_type -> temporal.api.cloud.cloudservice.v1.DeleteUserGroupResponse + 60, // 60: temporal.api.cloud.cloudservice.v1.CloudService.SetUserGroupNamespaceAccess:output_type -> temporal.api.cloud.cloudservice.v1.SetUserGroupNamespaceAccessResponse + 61, // 61: temporal.api.cloud.cloudservice.v1.CloudService.CreateServiceAccount:output_type -> temporal.api.cloud.cloudservice.v1.CreateServiceAccountResponse + 62, // 62: temporal.api.cloud.cloudservice.v1.CloudService.GetServiceAccount:output_type -> temporal.api.cloud.cloudservice.v1.GetServiceAccountResponse + 63, // 63: temporal.api.cloud.cloudservice.v1.CloudService.GetServiceAccounts:output_type -> temporal.api.cloud.cloudservice.v1.GetServiceAccountsResponse + 64, // 64: temporal.api.cloud.cloudservice.v1.CloudService.UpdateServiceAccount:output_type -> temporal.api.cloud.cloudservice.v1.UpdateServiceAccountResponse + 65, // 65: temporal.api.cloud.cloudservice.v1.CloudService.DeleteServiceAccount:output_type -> temporal.api.cloud.cloudservice.v1.DeleteServiceAccountResponse + 33, // [33:66] is the sub-list for method output_type + 0, // [0:33] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_cloud_cloudservice_v1_service_proto_init() } +func file_temporal_api_cloud_cloudservice_v1_service_proto_init() { + if File_temporal_api_cloud_cloudservice_v1_service_proto != nil { + return + } + file_temporal_api_cloud_cloudservice_v1_request_response_proto_init() + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_cloud_cloudservice_v1_service_proto_rawDesc, + NumEnums: 0, + NumMessages: 0, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_temporal_api_cloud_cloudservice_v1_service_proto_goTypes, + DependencyIndexes: file_temporal_api_cloud_cloudservice_v1_service_proto_depIdxs, + }.Build() + File_temporal_api_cloud_cloudservice_v1_service_proto = out.File + file_temporal_api_cloud_cloudservice_v1_service_proto_rawDesc = nil + file_temporal_api_cloud_cloudservice_v1_service_proto_goTypes = nil + file_temporal_api_cloud_cloudservice_v1_service_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/cloud/cloudservice/v1/service.pb.gw.go b/vendor/go.temporal.io/api/cloud/cloudservice/v1/service.pb.gw.go new file mode 100644 index 00000000000..0e8dce99507 --- /dev/null +++ b/vendor/go.temporal.io/api/cloud/cloudservice/v1/service.pb.gw.go @@ -0,0 +1,3474 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-grpc-gateway. DO NOT EDIT. +// source: temporal/api/cloud/cloudservice/v1/service.proto + +/* +Package cloudservice is a reverse proxy. + +It translates gRPC into RESTful JSON APIs. +*/ +package cloudservice + +import ( + "context" + "io" + "net/http" + + "github.com/grpc-ecosystem/grpc-gateway/v2/runtime" + "github.com/grpc-ecosystem/grpc-gateway/v2/utilities" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/grpclog" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/proto" +) + +// Suppress "imported and not used" errors +var _ codes.Code +var _ io.Reader +var _ status.Status +var _ = runtime.String +var _ = utilities.NewDoubleArray +var _ = metadata.Join + +var ( + filter_CloudService_GetUsers_0 = &utilities.DoubleArray{Encoding: map[string]int{}, Base: []int(nil), Check: []int(nil)} +) + +func request_CloudService_GetUsers_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetUsersRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_GetUsers_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetUsers(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_GetUsers_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetUsersRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_GetUsers_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetUsers(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_GetUser_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetUserRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["user_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "user_id") + } + + protoReq.UserId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "user_id", err) + } + + msg, err := client.GetUser(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_GetUser_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetUserRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["user_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "user_id") + } + + protoReq.UserId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "user_id", err) + } + + msg, err := server.GetUser(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_CreateUser_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateUserRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.CreateUser(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_CreateUser_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateUserRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.CreateUser(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_UpdateUser_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateUserRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["user_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "user_id") + } + + protoReq.UserId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "user_id", err) + } + + msg, err := client.UpdateUser(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_UpdateUser_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateUserRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["user_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "user_id") + } + + protoReq.UserId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "user_id", err) + } + + msg, err := server.UpdateUser(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_CloudService_DeleteUser_0 = &utilities.DoubleArray{Encoding: map[string]int{"user_id": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_CloudService_DeleteUser_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteUserRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["user_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "user_id") + } + + protoReq.UserId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "user_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_DeleteUser_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DeleteUser(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_DeleteUser_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteUserRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["user_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "user_id") + } + + protoReq.UserId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "user_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_DeleteUser_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DeleteUser(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_SetUserNamespaceAccess_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq SetUserNamespaceAccessRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["user_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "user_id") + } + + protoReq.UserId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "user_id", err) + } + + msg, err := client.SetUserNamespaceAccess(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_SetUserNamespaceAccess_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq SetUserNamespaceAccessRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["user_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "user_id") + } + + protoReq.UserId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "user_id", err) + } + + msg, err := server.SetUserNamespaceAccess(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_GetAsyncOperation_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetAsyncOperationRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["async_operation_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "async_operation_id") + } + + protoReq.AsyncOperationId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "async_operation_id", err) + } + + msg, err := client.GetAsyncOperation(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_GetAsyncOperation_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetAsyncOperationRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["async_operation_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "async_operation_id") + } + + protoReq.AsyncOperationId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "async_operation_id", err) + } + + msg, err := server.GetAsyncOperation(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_CreateNamespace_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateNamespaceRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.CreateNamespace(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_CreateNamespace_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateNamespaceRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.CreateNamespace(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_CloudService_GetNamespaces_0 = &utilities.DoubleArray{Encoding: map[string]int{}, Base: []int(nil), Check: []int(nil)} +) + +func request_CloudService_GetNamespaces_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetNamespacesRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_GetNamespaces_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetNamespaces(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_GetNamespaces_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetNamespacesRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_GetNamespaces_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetNamespaces(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_GetNamespace_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetNamespaceRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.GetNamespace(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_GetNamespace_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetNamespaceRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.GetNamespace(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_UpdateNamespace_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateNamespaceRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.UpdateNamespace(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_UpdateNamespace_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateNamespaceRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.UpdateNamespace(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_RenameCustomSearchAttribute_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RenameCustomSearchAttributeRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RenameCustomSearchAttribute(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_RenameCustomSearchAttribute_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RenameCustomSearchAttributeRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RenameCustomSearchAttribute(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_CloudService_DeleteNamespace_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_CloudService_DeleteNamespace_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteNamespaceRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_DeleteNamespace_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DeleteNamespace(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_DeleteNamespace_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteNamespaceRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_DeleteNamespace_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DeleteNamespace(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_FailoverNamespaceRegion_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq FailoverNamespaceRegionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.FailoverNamespaceRegion(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_FailoverNamespaceRegion_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq FailoverNamespaceRegionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.FailoverNamespaceRegion(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_AddNamespaceRegion_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq AddNamespaceRegionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.AddNamespaceRegion(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_AddNamespaceRegion_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq AddNamespaceRegionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.AddNamespaceRegion(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_GetRegions_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetRegionsRequest + var metadata runtime.ServerMetadata + + msg, err := client.GetRegions(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_GetRegions_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetRegionsRequest + var metadata runtime.ServerMetadata + + msg, err := server.GetRegions(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_GetRegion_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetRegionRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["region"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "region") + } + + protoReq.Region, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "region", err) + } + + msg, err := client.GetRegion(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_GetRegion_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetRegionRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["region"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "region") + } + + protoReq.Region, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "region", err) + } + + msg, err := server.GetRegion(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_CloudService_GetApiKeys_0 = &utilities.DoubleArray{Encoding: map[string]int{}, Base: []int(nil), Check: []int(nil)} +) + +func request_CloudService_GetApiKeys_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetApiKeysRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_GetApiKeys_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetApiKeys(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_GetApiKeys_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetApiKeysRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_GetApiKeys_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetApiKeys(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_GetApiKey_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetApiKeyRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["key_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "key_id") + } + + protoReq.KeyId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "key_id", err) + } + + msg, err := client.GetApiKey(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_GetApiKey_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetApiKeyRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["key_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "key_id") + } + + protoReq.KeyId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "key_id", err) + } + + msg, err := server.GetApiKey(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_CreateApiKey_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateApiKeyRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.CreateApiKey(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_CreateApiKey_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateApiKeyRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.CreateApiKey(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_UpdateApiKey_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateApiKeyRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["key_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "key_id") + } + + protoReq.KeyId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "key_id", err) + } + + msg, err := client.UpdateApiKey(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_UpdateApiKey_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateApiKeyRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["key_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "key_id") + } + + protoReq.KeyId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "key_id", err) + } + + msg, err := server.UpdateApiKey(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_CloudService_DeleteApiKey_0 = &utilities.DoubleArray{Encoding: map[string]int{"key_id": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_CloudService_DeleteApiKey_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteApiKeyRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["key_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "key_id") + } + + protoReq.KeyId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "key_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_DeleteApiKey_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DeleteApiKey(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_DeleteApiKey_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteApiKeyRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["key_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "key_id") + } + + protoReq.KeyId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "key_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_DeleteApiKey_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DeleteApiKey(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_CloudService_GetUserGroups_0 = &utilities.DoubleArray{Encoding: map[string]int{}, Base: []int(nil), Check: []int(nil)} +) + +func request_CloudService_GetUserGroups_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetUserGroupsRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_GetUserGroups_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetUserGroups(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_GetUserGroups_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetUserGroupsRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_GetUserGroups_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetUserGroups(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_GetUserGroup_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetUserGroupRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["group_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "group_id") + } + + protoReq.GroupId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "group_id", err) + } + + msg, err := client.GetUserGroup(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_GetUserGroup_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetUserGroupRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["group_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "group_id") + } + + protoReq.GroupId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "group_id", err) + } + + msg, err := server.GetUserGroup(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_CreateUserGroup_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateUserGroupRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.CreateUserGroup(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_CreateUserGroup_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateUserGroupRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.CreateUserGroup(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_UpdateUserGroup_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateUserGroupRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["group_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "group_id") + } + + protoReq.GroupId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "group_id", err) + } + + msg, err := client.UpdateUserGroup(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_UpdateUserGroup_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateUserGroupRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["group_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "group_id") + } + + protoReq.GroupId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "group_id", err) + } + + msg, err := server.UpdateUserGroup(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_CloudService_DeleteUserGroup_0 = &utilities.DoubleArray{Encoding: map[string]int{"group_id": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_CloudService_DeleteUserGroup_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteUserGroupRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["group_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "group_id") + } + + protoReq.GroupId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "group_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_DeleteUserGroup_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DeleteUserGroup(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_DeleteUserGroup_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteUserGroupRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["group_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "group_id") + } + + protoReq.GroupId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "group_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_DeleteUserGroup_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DeleteUserGroup(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_SetUserGroupNamespaceAccess_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq SetUserGroupNamespaceAccessRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["group_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "group_id") + } + + protoReq.GroupId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "group_id", err) + } + + msg, err := client.SetUserGroupNamespaceAccess(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_SetUserGroupNamespaceAccess_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq SetUserGroupNamespaceAccessRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["group_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "group_id") + } + + protoReq.GroupId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "group_id", err) + } + + msg, err := server.SetUserGroupNamespaceAccess(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_CreateServiceAccount_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateServiceAccountRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.CreateServiceAccount(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_CreateServiceAccount_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateServiceAccountRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.CreateServiceAccount(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_GetServiceAccount_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetServiceAccountRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["service_account_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "service_account_id") + } + + protoReq.ServiceAccountId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "service_account_id", err) + } + + msg, err := client.GetServiceAccount(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_GetServiceAccount_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetServiceAccountRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["service_account_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "service_account_id") + } + + protoReq.ServiceAccountId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "service_account_id", err) + } + + msg, err := server.GetServiceAccount(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_CloudService_GetServiceAccounts_0 = &utilities.DoubleArray{Encoding: map[string]int{}, Base: []int(nil), Check: []int(nil)} +) + +func request_CloudService_GetServiceAccounts_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetServiceAccountsRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_GetServiceAccounts_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetServiceAccounts(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_GetServiceAccounts_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetServiceAccountsRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_GetServiceAccounts_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetServiceAccounts(ctx, &protoReq) + return msg, metadata, err + +} + +func request_CloudService_UpdateServiceAccount_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateServiceAccountRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["service_account_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "service_account_id") + } + + protoReq.ServiceAccountId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "service_account_id", err) + } + + msg, err := client.UpdateServiceAccount(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_UpdateServiceAccount_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateServiceAccountRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["service_account_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "service_account_id") + } + + protoReq.ServiceAccountId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "service_account_id", err) + } + + msg, err := server.UpdateServiceAccount(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_CloudService_DeleteServiceAccount_0 = &utilities.DoubleArray{Encoding: map[string]int{"service_account_id": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_CloudService_DeleteServiceAccount_0(ctx context.Context, marshaler runtime.Marshaler, client CloudServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteServiceAccountRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["service_account_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "service_account_id") + } + + protoReq.ServiceAccountId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "service_account_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_DeleteServiceAccount_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DeleteServiceAccount(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_CloudService_DeleteServiceAccount_0(ctx context.Context, marshaler runtime.Marshaler, server CloudServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteServiceAccountRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["service_account_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "service_account_id") + } + + protoReq.ServiceAccountId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "service_account_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_CloudService_DeleteServiceAccount_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DeleteServiceAccount(ctx, &protoReq) + return msg, metadata, err + +} + +// RegisterCloudServiceHandlerServer registers the http handlers for service CloudService to "mux". +// UnaryRPC :call CloudServiceServer directly. +// StreamingRPC :currently unsupported pending https://github.com/grpc/grpc-go/issues/906. +// Note that using this registration option will cause many gRPC library features to stop working. Consider using RegisterCloudServiceHandlerFromEndpoint instead. +// GRPC interceptors will not work for this type of registration. To use interceptors, you must use the "runtime.WithMiddlewares" option in the "runtime.NewServeMux" call. +func RegisterCloudServiceHandlerServer(ctx context.Context, mux *runtime.ServeMux, server CloudServiceServer) error { + + mux.Handle("GET", pattern_CloudService_GetUsers_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetUsers", runtime.WithHTTPPathPattern("/cloud/users")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_GetUsers_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetUsers_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetUser_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetUser", runtime.WithHTTPPathPattern("/cloud/users/{user_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_GetUser_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetUser_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_CreateUser_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/CreateUser", runtime.WithHTTPPathPattern("/cloud/users")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_CreateUser_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_CreateUser_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_UpdateUser_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateUser", runtime.WithHTTPPathPattern("/cloud/users/{user_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_UpdateUser_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_UpdateUser_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_CloudService_DeleteUser_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteUser", runtime.WithHTTPPathPattern("/cloud/users/{user_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_DeleteUser_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_DeleteUser_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_SetUserNamespaceAccess_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/SetUserNamespaceAccess", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}/users/{user_id}/access")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_SetUserNamespaceAccess_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_SetUserNamespaceAccess_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetAsyncOperation_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetAsyncOperation", runtime.WithHTTPPathPattern("/cloud/operations/{async_operation_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_GetAsyncOperation_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetAsyncOperation_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_CreateNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/CreateNamespace", runtime.WithHTTPPathPattern("/cloud/namespaces")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_CreateNamespace_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_CreateNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetNamespaces_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetNamespaces", runtime.WithHTTPPathPattern("/cloud/namespaces")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_GetNamespaces_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetNamespaces_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetNamespace", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_GetNamespace_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_UpdateNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateNamespace", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_UpdateNamespace_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_UpdateNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_RenameCustomSearchAttribute_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/RenameCustomSearchAttribute", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}/rename-custom-search-attribute")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_RenameCustomSearchAttribute_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_RenameCustomSearchAttribute_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_CloudService_DeleteNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteNamespace", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_DeleteNamespace_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_DeleteNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_FailoverNamespaceRegion_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/FailoverNamespaceRegion", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}/failover-region")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_FailoverNamespaceRegion_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_FailoverNamespaceRegion_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_AddNamespaceRegion_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/AddNamespaceRegion", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}/add-region")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_AddNamespaceRegion_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_AddNamespaceRegion_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetRegions_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetRegions", runtime.WithHTTPPathPattern("/cloud/regions")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_GetRegions_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetRegions_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetRegion_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetRegion", runtime.WithHTTPPathPattern("/cloud/regions/{region}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_GetRegion_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetRegion_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetApiKeys_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetApiKeys", runtime.WithHTTPPathPattern("/cloud/api-keys")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_GetApiKeys_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetApiKeys_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetApiKey_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetApiKey", runtime.WithHTTPPathPattern("/cloud/api-keys/{key_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_GetApiKey_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetApiKey_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_CreateApiKey_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/CreateApiKey", runtime.WithHTTPPathPattern("/cloud/api-keys")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_CreateApiKey_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_CreateApiKey_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_UpdateApiKey_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateApiKey", runtime.WithHTTPPathPattern("/cloud/api-keys/{key_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_UpdateApiKey_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_UpdateApiKey_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_CloudService_DeleteApiKey_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteApiKey", runtime.WithHTTPPathPattern("/cloud/api-keys/{key_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_DeleteApiKey_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_DeleteApiKey_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetUserGroups_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetUserGroups", runtime.WithHTTPPathPattern("/cloud/user-groups")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_GetUserGroups_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetUserGroups_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetUserGroup_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetUserGroup", runtime.WithHTTPPathPattern("/cloud/user-groups/{group_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_GetUserGroup_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetUserGroup_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_CreateUserGroup_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/CreateUserGroup", runtime.WithHTTPPathPattern("/cloud/user-groups")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_CreateUserGroup_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_CreateUserGroup_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_UpdateUserGroup_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateUserGroup", runtime.WithHTTPPathPattern("/cloud/user-groups/{group_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_UpdateUserGroup_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_UpdateUserGroup_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_CloudService_DeleteUserGroup_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteUserGroup", runtime.WithHTTPPathPattern("/cloud/user-groups/{group_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_DeleteUserGroup_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_DeleteUserGroup_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_SetUserGroupNamespaceAccess_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/SetUserGroupNamespaceAccess", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}/user-groups/{group_id}/access")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_SetUserGroupNamespaceAccess_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_SetUserGroupNamespaceAccess_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_CreateServiceAccount_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/CreateServiceAccount", runtime.WithHTTPPathPattern("/cloud/service-accounts")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_CreateServiceAccount_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_CreateServiceAccount_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetServiceAccount_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetServiceAccount", runtime.WithHTTPPathPattern("/cloud/service-accounts/{service_account_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_GetServiceAccount_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetServiceAccount_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetServiceAccounts_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetServiceAccounts", runtime.WithHTTPPathPattern("/cloud/service-accounts")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_GetServiceAccounts_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetServiceAccounts_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_UpdateServiceAccount_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateServiceAccount", runtime.WithHTTPPathPattern("/cloud/service-accounts/{service_account_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_UpdateServiceAccount_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_UpdateServiceAccount_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_CloudService_DeleteServiceAccount_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteServiceAccount", runtime.WithHTTPPathPattern("/cloud/service-accounts/{service_account_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_CloudService_DeleteServiceAccount_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_DeleteServiceAccount_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + return nil +} + +// RegisterCloudServiceHandlerFromEndpoint is same as RegisterCloudServiceHandler but +// automatically dials to "endpoint" and closes the connection when "ctx" gets done. +func RegisterCloudServiceHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) { + conn, err := grpc.NewClient(endpoint, opts...) + if err != nil { + return err + } + defer func() { + if err != nil { + if cerr := conn.Close(); cerr != nil { + grpclog.Errorf("Failed to close conn to %s: %v", endpoint, cerr) + } + return + } + go func() { + <-ctx.Done() + if cerr := conn.Close(); cerr != nil { + grpclog.Errorf("Failed to close conn to %s: %v", endpoint, cerr) + } + }() + }() + + return RegisterCloudServiceHandler(ctx, mux, conn) +} + +// RegisterCloudServiceHandler registers the http handlers for service CloudService to "mux". +// The handlers forward requests to the grpc endpoint over "conn". +func RegisterCloudServiceHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error { + return RegisterCloudServiceHandlerClient(ctx, mux, NewCloudServiceClient(conn)) +} + +// RegisterCloudServiceHandlerClient registers the http handlers for service CloudService +// to "mux". The handlers forward requests to the grpc endpoint over the given implementation of "CloudServiceClient". +// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "CloudServiceClient" +// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in +// "CloudServiceClient" to call the correct interceptors. This client ignores the HTTP middlewares. +func RegisterCloudServiceHandlerClient(ctx context.Context, mux *runtime.ServeMux, client CloudServiceClient) error { + + mux.Handle("GET", pattern_CloudService_GetUsers_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetUsers", runtime.WithHTTPPathPattern("/cloud/users")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_GetUsers_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetUsers_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetUser_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetUser", runtime.WithHTTPPathPattern("/cloud/users/{user_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_GetUser_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetUser_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_CreateUser_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/CreateUser", runtime.WithHTTPPathPattern("/cloud/users")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_CreateUser_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_CreateUser_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_UpdateUser_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateUser", runtime.WithHTTPPathPattern("/cloud/users/{user_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_UpdateUser_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_UpdateUser_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_CloudService_DeleteUser_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteUser", runtime.WithHTTPPathPattern("/cloud/users/{user_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_DeleteUser_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_DeleteUser_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_SetUserNamespaceAccess_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/SetUserNamespaceAccess", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}/users/{user_id}/access")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_SetUserNamespaceAccess_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_SetUserNamespaceAccess_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetAsyncOperation_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetAsyncOperation", runtime.WithHTTPPathPattern("/cloud/operations/{async_operation_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_GetAsyncOperation_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetAsyncOperation_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_CreateNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/CreateNamespace", runtime.WithHTTPPathPattern("/cloud/namespaces")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_CreateNamespace_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_CreateNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetNamespaces_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetNamespaces", runtime.WithHTTPPathPattern("/cloud/namespaces")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_GetNamespaces_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetNamespaces_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetNamespace", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_GetNamespace_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_UpdateNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateNamespace", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_UpdateNamespace_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_UpdateNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_RenameCustomSearchAttribute_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/RenameCustomSearchAttribute", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}/rename-custom-search-attribute")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_RenameCustomSearchAttribute_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_RenameCustomSearchAttribute_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_CloudService_DeleteNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteNamespace", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_DeleteNamespace_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_DeleteNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_FailoverNamespaceRegion_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/FailoverNamespaceRegion", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}/failover-region")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_FailoverNamespaceRegion_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_FailoverNamespaceRegion_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_AddNamespaceRegion_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/AddNamespaceRegion", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}/add-region")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_AddNamespaceRegion_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_AddNamespaceRegion_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetRegions_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetRegions", runtime.WithHTTPPathPattern("/cloud/regions")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_GetRegions_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetRegions_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetRegion_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetRegion", runtime.WithHTTPPathPattern("/cloud/regions/{region}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_GetRegion_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetRegion_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetApiKeys_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetApiKeys", runtime.WithHTTPPathPattern("/cloud/api-keys")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_GetApiKeys_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetApiKeys_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetApiKey_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetApiKey", runtime.WithHTTPPathPattern("/cloud/api-keys/{key_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_GetApiKey_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetApiKey_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_CreateApiKey_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/CreateApiKey", runtime.WithHTTPPathPattern("/cloud/api-keys")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_CreateApiKey_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_CreateApiKey_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_UpdateApiKey_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateApiKey", runtime.WithHTTPPathPattern("/cloud/api-keys/{key_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_UpdateApiKey_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_UpdateApiKey_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_CloudService_DeleteApiKey_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteApiKey", runtime.WithHTTPPathPattern("/cloud/api-keys/{key_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_DeleteApiKey_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_DeleteApiKey_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetUserGroups_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetUserGroups", runtime.WithHTTPPathPattern("/cloud/user-groups")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_GetUserGroups_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetUserGroups_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetUserGroup_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetUserGroup", runtime.WithHTTPPathPattern("/cloud/user-groups/{group_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_GetUserGroup_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetUserGroup_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_CreateUserGroup_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/CreateUserGroup", runtime.WithHTTPPathPattern("/cloud/user-groups")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_CreateUserGroup_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_CreateUserGroup_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_UpdateUserGroup_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateUserGroup", runtime.WithHTTPPathPattern("/cloud/user-groups/{group_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_UpdateUserGroup_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_UpdateUserGroup_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_CloudService_DeleteUserGroup_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteUserGroup", runtime.WithHTTPPathPattern("/cloud/user-groups/{group_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_DeleteUserGroup_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_DeleteUserGroup_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_SetUserGroupNamespaceAccess_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/SetUserGroupNamespaceAccess", runtime.WithHTTPPathPattern("/cloud/namespaces/{namespace}/user-groups/{group_id}/access")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_SetUserGroupNamespaceAccess_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_SetUserGroupNamespaceAccess_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_CreateServiceAccount_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/CreateServiceAccount", runtime.WithHTTPPathPattern("/cloud/service-accounts")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_CreateServiceAccount_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_CreateServiceAccount_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetServiceAccount_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetServiceAccount", runtime.WithHTTPPathPattern("/cloud/service-accounts/{service_account_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_GetServiceAccount_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetServiceAccount_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_CloudService_GetServiceAccounts_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/GetServiceAccounts", runtime.WithHTTPPathPattern("/cloud/service-accounts")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_GetServiceAccounts_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_GetServiceAccounts_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_CloudService_UpdateServiceAccount_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateServiceAccount", runtime.WithHTTPPathPattern("/cloud/service-accounts/{service_account_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_UpdateServiceAccount_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_UpdateServiceAccount_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_CloudService_DeleteServiceAccount_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteServiceAccount", runtime.WithHTTPPathPattern("/cloud/service-accounts/{service_account_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_CloudService_DeleteServiceAccount_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_CloudService_DeleteServiceAccount_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + return nil +} + +var ( + pattern_CloudService_GetUsers_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"cloud", "users"}, "")) + + pattern_CloudService_GetUser_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "users", "user_id"}, "")) + + pattern_CloudService_CreateUser_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"cloud", "users"}, "")) + + pattern_CloudService_UpdateUser_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "users", "user_id"}, "")) + + pattern_CloudService_DeleteUser_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "users", "user_id"}, "")) + + pattern_CloudService_SetUserNamespaceAccess_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2, 2, 3, 1, 0, 4, 1, 5, 4, 2, 5}, []string{"cloud", "namespaces", "namespace", "users", "user_id", "access"}, "")) + + pattern_CloudService_GetAsyncOperation_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "operations", "async_operation_id"}, "")) + + pattern_CloudService_CreateNamespace_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"cloud", "namespaces"}, "")) + + pattern_CloudService_GetNamespaces_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"cloud", "namespaces"}, "")) + + pattern_CloudService_GetNamespace_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "namespaces", "namespace"}, "")) + + pattern_CloudService_UpdateNamespace_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "namespaces", "namespace"}, "")) + + pattern_CloudService_RenameCustomSearchAttribute_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2, 2, 3}, []string{"cloud", "namespaces", "namespace", "rename-custom-search-attribute"}, "")) + + pattern_CloudService_DeleteNamespace_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "namespaces", "namespace"}, "")) + + pattern_CloudService_FailoverNamespaceRegion_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2, 2, 3}, []string{"cloud", "namespaces", "namespace", "failover-region"}, "")) + + pattern_CloudService_AddNamespaceRegion_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2, 2, 3}, []string{"cloud", "namespaces", "namespace", "add-region"}, "")) + + pattern_CloudService_GetRegions_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"cloud", "regions"}, "")) + + pattern_CloudService_GetRegion_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "regions", "region"}, "")) + + pattern_CloudService_GetApiKeys_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"cloud", "api-keys"}, "")) + + pattern_CloudService_GetApiKey_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "api-keys", "key_id"}, "")) + + pattern_CloudService_CreateApiKey_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"cloud", "api-keys"}, "")) + + pattern_CloudService_UpdateApiKey_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "api-keys", "key_id"}, "")) + + pattern_CloudService_DeleteApiKey_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "api-keys", "key_id"}, "")) + + pattern_CloudService_GetUserGroups_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"cloud", "user-groups"}, "")) + + pattern_CloudService_GetUserGroup_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "user-groups", "group_id"}, "")) + + pattern_CloudService_CreateUserGroup_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"cloud", "user-groups"}, "")) + + pattern_CloudService_UpdateUserGroup_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "user-groups", "group_id"}, "")) + + pattern_CloudService_DeleteUserGroup_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "user-groups", "group_id"}, "")) + + pattern_CloudService_SetUserGroupNamespaceAccess_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2, 2, 3, 1, 0, 4, 1, 5, 4, 2, 5}, []string{"cloud", "namespaces", "namespace", "user-groups", "group_id", "access"}, "")) + + pattern_CloudService_CreateServiceAccount_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"cloud", "service-accounts"}, "")) + + pattern_CloudService_GetServiceAccount_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "service-accounts", "service_account_id"}, "")) + + pattern_CloudService_GetServiceAccounts_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"cloud", "service-accounts"}, "")) + + pattern_CloudService_UpdateServiceAccount_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "service-accounts", "service_account_id"}, "")) + + pattern_CloudService_DeleteServiceAccount_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cloud", "service-accounts", "service_account_id"}, "")) +) + +var ( + forward_CloudService_GetUsers_0 = runtime.ForwardResponseMessage + + forward_CloudService_GetUser_0 = runtime.ForwardResponseMessage + + forward_CloudService_CreateUser_0 = runtime.ForwardResponseMessage + + forward_CloudService_UpdateUser_0 = runtime.ForwardResponseMessage + + forward_CloudService_DeleteUser_0 = runtime.ForwardResponseMessage + + forward_CloudService_SetUserNamespaceAccess_0 = runtime.ForwardResponseMessage + + forward_CloudService_GetAsyncOperation_0 = runtime.ForwardResponseMessage + + forward_CloudService_CreateNamespace_0 = runtime.ForwardResponseMessage + + forward_CloudService_GetNamespaces_0 = runtime.ForwardResponseMessage + + forward_CloudService_GetNamespace_0 = runtime.ForwardResponseMessage + + forward_CloudService_UpdateNamespace_0 = runtime.ForwardResponseMessage + + forward_CloudService_RenameCustomSearchAttribute_0 = runtime.ForwardResponseMessage + + forward_CloudService_DeleteNamespace_0 = runtime.ForwardResponseMessage + + forward_CloudService_FailoverNamespaceRegion_0 = runtime.ForwardResponseMessage + + forward_CloudService_AddNamespaceRegion_0 = runtime.ForwardResponseMessage + + forward_CloudService_GetRegions_0 = runtime.ForwardResponseMessage + + forward_CloudService_GetRegion_0 = runtime.ForwardResponseMessage + + forward_CloudService_GetApiKeys_0 = runtime.ForwardResponseMessage + + forward_CloudService_GetApiKey_0 = runtime.ForwardResponseMessage + + forward_CloudService_CreateApiKey_0 = runtime.ForwardResponseMessage + + forward_CloudService_UpdateApiKey_0 = runtime.ForwardResponseMessage + + forward_CloudService_DeleteApiKey_0 = runtime.ForwardResponseMessage + + forward_CloudService_GetUserGroups_0 = runtime.ForwardResponseMessage + + forward_CloudService_GetUserGroup_0 = runtime.ForwardResponseMessage + + forward_CloudService_CreateUserGroup_0 = runtime.ForwardResponseMessage + + forward_CloudService_UpdateUserGroup_0 = runtime.ForwardResponseMessage + + forward_CloudService_DeleteUserGroup_0 = runtime.ForwardResponseMessage + + forward_CloudService_SetUserGroupNamespaceAccess_0 = runtime.ForwardResponseMessage + + forward_CloudService_CreateServiceAccount_0 = runtime.ForwardResponseMessage + + forward_CloudService_GetServiceAccount_0 = runtime.ForwardResponseMessage + + forward_CloudService_GetServiceAccounts_0 = runtime.ForwardResponseMessage + + forward_CloudService_UpdateServiceAccount_0 = runtime.ForwardResponseMessage + + forward_CloudService_DeleteServiceAccount_0 = runtime.ForwardResponseMessage +) diff --git a/vendor/go.temporal.io/api/cloud/cloudservice/v1/service_grpc.pb.go b/vendor/go.temporal.io/api/cloud/cloudservice/v1/service_grpc.pb.go new file mode 100644 index 00000000000..f3bc2db7420 --- /dev/null +++ b/vendor/go.temporal.io/api/cloud/cloudservice/v1/service_grpc.pb.go @@ -0,0 +1,1432 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// plugins: +// - protoc-gen-go-grpc +// - protoc +// source: temporal/api/cloud/cloudservice/v1/service.proto + +package cloudservice + +import ( + context "context" + + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.64.0 or later. +const _ = grpc.SupportPackageIsVersion9 + +const ( + CloudService_GetUsers_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/GetUsers" + CloudService_GetUser_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/GetUser" + CloudService_CreateUser_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/CreateUser" + CloudService_UpdateUser_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateUser" + CloudService_DeleteUser_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteUser" + CloudService_SetUserNamespaceAccess_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/SetUserNamespaceAccess" + CloudService_GetAsyncOperation_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/GetAsyncOperation" + CloudService_CreateNamespace_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/CreateNamespace" + CloudService_GetNamespaces_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/GetNamespaces" + CloudService_GetNamespace_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/GetNamespace" + CloudService_UpdateNamespace_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateNamespace" + CloudService_RenameCustomSearchAttribute_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/RenameCustomSearchAttribute" + CloudService_DeleteNamespace_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteNamespace" + CloudService_FailoverNamespaceRegion_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/FailoverNamespaceRegion" + CloudService_AddNamespaceRegion_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/AddNamespaceRegion" + CloudService_GetRegions_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/GetRegions" + CloudService_GetRegion_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/GetRegion" + CloudService_GetApiKeys_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/GetApiKeys" + CloudService_GetApiKey_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/GetApiKey" + CloudService_CreateApiKey_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/CreateApiKey" + CloudService_UpdateApiKey_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateApiKey" + CloudService_DeleteApiKey_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteApiKey" + CloudService_GetUserGroups_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/GetUserGroups" + CloudService_GetUserGroup_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/GetUserGroup" + CloudService_CreateUserGroup_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/CreateUserGroup" + CloudService_UpdateUserGroup_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateUserGroup" + CloudService_DeleteUserGroup_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteUserGroup" + CloudService_SetUserGroupNamespaceAccess_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/SetUserGroupNamespaceAccess" + CloudService_CreateServiceAccount_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/CreateServiceAccount" + CloudService_GetServiceAccount_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/GetServiceAccount" + CloudService_GetServiceAccounts_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/GetServiceAccounts" + CloudService_UpdateServiceAccount_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/UpdateServiceAccount" + CloudService_DeleteServiceAccount_FullMethodName = "/temporal.api.cloud.cloudservice.v1.CloudService/DeleteServiceAccount" +) + +// CloudServiceClient is the client API for CloudService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// WARNING: This service is currently experimental and may change in +// incompatible ways. +type CloudServiceClient interface { + // Gets all known users + GetUsers(ctx context.Context, in *GetUsersRequest, opts ...grpc.CallOption) (*GetUsersResponse, error) + // Get a user + GetUser(ctx context.Context, in *GetUserRequest, opts ...grpc.CallOption) (*GetUserResponse, error) + // Create a user + CreateUser(ctx context.Context, in *CreateUserRequest, opts ...grpc.CallOption) (*CreateUserResponse, error) + // Update a user + UpdateUser(ctx context.Context, in *UpdateUserRequest, opts ...grpc.CallOption) (*UpdateUserResponse, error) + // Delete a user + DeleteUser(ctx context.Context, in *DeleteUserRequest, opts ...grpc.CallOption) (*DeleteUserResponse, error) + // Set a user's access to a namespace + SetUserNamespaceAccess(ctx context.Context, in *SetUserNamespaceAccessRequest, opts ...grpc.CallOption) (*SetUserNamespaceAccessResponse, error) + // Get the latest information on an async operation + GetAsyncOperation(ctx context.Context, in *GetAsyncOperationRequest, opts ...grpc.CallOption) (*GetAsyncOperationResponse, error) + // Create a new namespace + CreateNamespace(ctx context.Context, in *CreateNamespaceRequest, opts ...grpc.CallOption) (*CreateNamespaceResponse, error) + // Get all namespaces + GetNamespaces(ctx context.Context, in *GetNamespacesRequest, opts ...grpc.CallOption) (*GetNamespacesResponse, error) + // Get a namespace + GetNamespace(ctx context.Context, in *GetNamespaceRequest, opts ...grpc.CallOption) (*GetNamespaceResponse, error) + // Update a namespace + UpdateNamespace(ctx context.Context, in *UpdateNamespaceRequest, opts ...grpc.CallOption) (*UpdateNamespaceResponse, error) + // Rename an existing customer search attribute + RenameCustomSearchAttribute(ctx context.Context, in *RenameCustomSearchAttributeRequest, opts ...grpc.CallOption) (*RenameCustomSearchAttributeResponse, error) + // Delete a namespace + DeleteNamespace(ctx context.Context, in *DeleteNamespaceRequest, opts ...grpc.CallOption) (*DeleteNamespaceResponse, error) + // Failover a multi-region namespace + FailoverNamespaceRegion(ctx context.Context, in *FailoverNamespaceRegionRequest, opts ...grpc.CallOption) (*FailoverNamespaceRegionResponse, error) + // Add a new region to a namespace + AddNamespaceRegion(ctx context.Context, in *AddNamespaceRegionRequest, opts ...grpc.CallOption) (*AddNamespaceRegionResponse, error) + // Get all regions + GetRegions(ctx context.Context, in *GetRegionsRequest, opts ...grpc.CallOption) (*GetRegionsResponse, error) + // Get a region + GetRegion(ctx context.Context, in *GetRegionRequest, opts ...grpc.CallOption) (*GetRegionResponse, error) + // Get all known API keys + GetApiKeys(ctx context.Context, in *GetApiKeysRequest, opts ...grpc.CallOption) (*GetApiKeysResponse, error) + // Get an API key + GetApiKey(ctx context.Context, in *GetApiKeyRequest, opts ...grpc.CallOption) (*GetApiKeyResponse, error) + // Create an API key + CreateApiKey(ctx context.Context, in *CreateApiKeyRequest, opts ...grpc.CallOption) (*CreateApiKeyResponse, error) + // Update an API key + UpdateApiKey(ctx context.Context, in *UpdateApiKeyRequest, opts ...grpc.CallOption) (*UpdateApiKeyResponse, error) + // Delete an API key + DeleteApiKey(ctx context.Context, in *DeleteApiKeyRequest, opts ...grpc.CallOption) (*DeleteApiKeyResponse, error) + // Get all user groups + GetUserGroups(ctx context.Context, in *GetUserGroupsRequest, opts ...grpc.CallOption) (*GetUserGroupsResponse, error) + // Get a user group + GetUserGroup(ctx context.Context, in *GetUserGroupRequest, opts ...grpc.CallOption) (*GetUserGroupResponse, error) + // Create new a user group + CreateUserGroup(ctx context.Context, in *CreateUserGroupRequest, opts ...grpc.CallOption) (*CreateUserGroupResponse, error) + // Update a user group + UpdateUserGroup(ctx context.Context, in *UpdateUserGroupRequest, opts ...grpc.CallOption) (*UpdateUserGroupResponse, error) + // Delete a user group + DeleteUserGroup(ctx context.Context, in *DeleteUserGroupRequest, opts ...grpc.CallOption) (*DeleteUserGroupResponse, error) + // Set a user group's access to a namespace + SetUserGroupNamespaceAccess(ctx context.Context, in *SetUserGroupNamespaceAccessRequest, opts ...grpc.CallOption) (*SetUserGroupNamespaceAccessResponse, error) + // Create a service account. + CreateServiceAccount(ctx context.Context, in *CreateServiceAccountRequest, opts ...grpc.CallOption) (*CreateServiceAccountResponse, error) + // Get a service account. + GetServiceAccount(ctx context.Context, in *GetServiceAccountRequest, opts ...grpc.CallOption) (*GetServiceAccountResponse, error) + // Get service accounts. + GetServiceAccounts(ctx context.Context, in *GetServiceAccountsRequest, opts ...grpc.CallOption) (*GetServiceAccountsResponse, error) + // Update a service account. + UpdateServiceAccount(ctx context.Context, in *UpdateServiceAccountRequest, opts ...grpc.CallOption) (*UpdateServiceAccountResponse, error) + // Delete a service account. + DeleteServiceAccount(ctx context.Context, in *DeleteServiceAccountRequest, opts ...grpc.CallOption) (*DeleteServiceAccountResponse, error) +} + +type cloudServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewCloudServiceClient(cc grpc.ClientConnInterface) CloudServiceClient { + return &cloudServiceClient{cc} +} + +func (c *cloudServiceClient) GetUsers(ctx context.Context, in *GetUsersRequest, opts ...grpc.CallOption) (*GetUsersResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetUsersResponse) + err := c.cc.Invoke(ctx, CloudService_GetUsers_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) GetUser(ctx context.Context, in *GetUserRequest, opts ...grpc.CallOption) (*GetUserResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetUserResponse) + err := c.cc.Invoke(ctx, CloudService_GetUser_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) CreateUser(ctx context.Context, in *CreateUserRequest, opts ...grpc.CallOption) (*CreateUserResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(CreateUserResponse) + err := c.cc.Invoke(ctx, CloudService_CreateUser_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) UpdateUser(ctx context.Context, in *UpdateUserRequest, opts ...grpc.CallOption) (*UpdateUserResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(UpdateUserResponse) + err := c.cc.Invoke(ctx, CloudService_UpdateUser_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) DeleteUser(ctx context.Context, in *DeleteUserRequest, opts ...grpc.CallOption) (*DeleteUserResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DeleteUserResponse) + err := c.cc.Invoke(ctx, CloudService_DeleteUser_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) SetUserNamespaceAccess(ctx context.Context, in *SetUserNamespaceAccessRequest, opts ...grpc.CallOption) (*SetUserNamespaceAccessResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(SetUserNamespaceAccessResponse) + err := c.cc.Invoke(ctx, CloudService_SetUserNamespaceAccess_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) GetAsyncOperation(ctx context.Context, in *GetAsyncOperationRequest, opts ...grpc.CallOption) (*GetAsyncOperationResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetAsyncOperationResponse) + err := c.cc.Invoke(ctx, CloudService_GetAsyncOperation_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) CreateNamespace(ctx context.Context, in *CreateNamespaceRequest, opts ...grpc.CallOption) (*CreateNamespaceResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(CreateNamespaceResponse) + err := c.cc.Invoke(ctx, CloudService_CreateNamespace_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) GetNamespaces(ctx context.Context, in *GetNamespacesRequest, opts ...grpc.CallOption) (*GetNamespacesResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetNamespacesResponse) + err := c.cc.Invoke(ctx, CloudService_GetNamespaces_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) GetNamespace(ctx context.Context, in *GetNamespaceRequest, opts ...grpc.CallOption) (*GetNamespaceResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetNamespaceResponse) + err := c.cc.Invoke(ctx, CloudService_GetNamespace_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) UpdateNamespace(ctx context.Context, in *UpdateNamespaceRequest, opts ...grpc.CallOption) (*UpdateNamespaceResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(UpdateNamespaceResponse) + err := c.cc.Invoke(ctx, CloudService_UpdateNamespace_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) RenameCustomSearchAttribute(ctx context.Context, in *RenameCustomSearchAttributeRequest, opts ...grpc.CallOption) (*RenameCustomSearchAttributeResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RenameCustomSearchAttributeResponse) + err := c.cc.Invoke(ctx, CloudService_RenameCustomSearchAttribute_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) DeleteNamespace(ctx context.Context, in *DeleteNamespaceRequest, opts ...grpc.CallOption) (*DeleteNamespaceResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DeleteNamespaceResponse) + err := c.cc.Invoke(ctx, CloudService_DeleteNamespace_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) FailoverNamespaceRegion(ctx context.Context, in *FailoverNamespaceRegionRequest, opts ...grpc.CallOption) (*FailoverNamespaceRegionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(FailoverNamespaceRegionResponse) + err := c.cc.Invoke(ctx, CloudService_FailoverNamespaceRegion_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) AddNamespaceRegion(ctx context.Context, in *AddNamespaceRegionRequest, opts ...grpc.CallOption) (*AddNamespaceRegionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(AddNamespaceRegionResponse) + err := c.cc.Invoke(ctx, CloudService_AddNamespaceRegion_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) GetRegions(ctx context.Context, in *GetRegionsRequest, opts ...grpc.CallOption) (*GetRegionsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetRegionsResponse) + err := c.cc.Invoke(ctx, CloudService_GetRegions_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) GetRegion(ctx context.Context, in *GetRegionRequest, opts ...grpc.CallOption) (*GetRegionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetRegionResponse) + err := c.cc.Invoke(ctx, CloudService_GetRegion_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) GetApiKeys(ctx context.Context, in *GetApiKeysRequest, opts ...grpc.CallOption) (*GetApiKeysResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetApiKeysResponse) + err := c.cc.Invoke(ctx, CloudService_GetApiKeys_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) GetApiKey(ctx context.Context, in *GetApiKeyRequest, opts ...grpc.CallOption) (*GetApiKeyResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetApiKeyResponse) + err := c.cc.Invoke(ctx, CloudService_GetApiKey_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) CreateApiKey(ctx context.Context, in *CreateApiKeyRequest, opts ...grpc.CallOption) (*CreateApiKeyResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(CreateApiKeyResponse) + err := c.cc.Invoke(ctx, CloudService_CreateApiKey_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) UpdateApiKey(ctx context.Context, in *UpdateApiKeyRequest, opts ...grpc.CallOption) (*UpdateApiKeyResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(UpdateApiKeyResponse) + err := c.cc.Invoke(ctx, CloudService_UpdateApiKey_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) DeleteApiKey(ctx context.Context, in *DeleteApiKeyRequest, opts ...grpc.CallOption) (*DeleteApiKeyResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DeleteApiKeyResponse) + err := c.cc.Invoke(ctx, CloudService_DeleteApiKey_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) GetUserGroups(ctx context.Context, in *GetUserGroupsRequest, opts ...grpc.CallOption) (*GetUserGroupsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetUserGroupsResponse) + err := c.cc.Invoke(ctx, CloudService_GetUserGroups_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) GetUserGroup(ctx context.Context, in *GetUserGroupRequest, opts ...grpc.CallOption) (*GetUserGroupResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetUserGroupResponse) + err := c.cc.Invoke(ctx, CloudService_GetUserGroup_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) CreateUserGroup(ctx context.Context, in *CreateUserGroupRequest, opts ...grpc.CallOption) (*CreateUserGroupResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(CreateUserGroupResponse) + err := c.cc.Invoke(ctx, CloudService_CreateUserGroup_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) UpdateUserGroup(ctx context.Context, in *UpdateUserGroupRequest, opts ...grpc.CallOption) (*UpdateUserGroupResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(UpdateUserGroupResponse) + err := c.cc.Invoke(ctx, CloudService_UpdateUserGroup_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) DeleteUserGroup(ctx context.Context, in *DeleteUserGroupRequest, opts ...grpc.CallOption) (*DeleteUserGroupResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DeleteUserGroupResponse) + err := c.cc.Invoke(ctx, CloudService_DeleteUserGroup_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) SetUserGroupNamespaceAccess(ctx context.Context, in *SetUserGroupNamespaceAccessRequest, opts ...grpc.CallOption) (*SetUserGroupNamespaceAccessResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(SetUserGroupNamespaceAccessResponse) + err := c.cc.Invoke(ctx, CloudService_SetUserGroupNamespaceAccess_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) CreateServiceAccount(ctx context.Context, in *CreateServiceAccountRequest, opts ...grpc.CallOption) (*CreateServiceAccountResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(CreateServiceAccountResponse) + err := c.cc.Invoke(ctx, CloudService_CreateServiceAccount_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) GetServiceAccount(ctx context.Context, in *GetServiceAccountRequest, opts ...grpc.CallOption) (*GetServiceAccountResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetServiceAccountResponse) + err := c.cc.Invoke(ctx, CloudService_GetServiceAccount_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) GetServiceAccounts(ctx context.Context, in *GetServiceAccountsRequest, opts ...grpc.CallOption) (*GetServiceAccountsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetServiceAccountsResponse) + err := c.cc.Invoke(ctx, CloudService_GetServiceAccounts_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) UpdateServiceAccount(ctx context.Context, in *UpdateServiceAccountRequest, opts ...grpc.CallOption) (*UpdateServiceAccountResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(UpdateServiceAccountResponse) + err := c.cc.Invoke(ctx, CloudService_UpdateServiceAccount_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *cloudServiceClient) DeleteServiceAccount(ctx context.Context, in *DeleteServiceAccountRequest, opts ...grpc.CallOption) (*DeleteServiceAccountResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DeleteServiceAccountResponse) + err := c.cc.Invoke(ctx, CloudService_DeleteServiceAccount_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +// CloudServiceServer is the server API for CloudService service. +// All implementations must embed UnimplementedCloudServiceServer +// for forward compatibility. +// +// WARNING: This service is currently experimental and may change in +// incompatible ways. +type CloudServiceServer interface { + // Gets all known users + GetUsers(context.Context, *GetUsersRequest) (*GetUsersResponse, error) + // Get a user + GetUser(context.Context, *GetUserRequest) (*GetUserResponse, error) + // Create a user + CreateUser(context.Context, *CreateUserRequest) (*CreateUserResponse, error) + // Update a user + UpdateUser(context.Context, *UpdateUserRequest) (*UpdateUserResponse, error) + // Delete a user + DeleteUser(context.Context, *DeleteUserRequest) (*DeleteUserResponse, error) + // Set a user's access to a namespace + SetUserNamespaceAccess(context.Context, *SetUserNamespaceAccessRequest) (*SetUserNamespaceAccessResponse, error) + // Get the latest information on an async operation + GetAsyncOperation(context.Context, *GetAsyncOperationRequest) (*GetAsyncOperationResponse, error) + // Create a new namespace + CreateNamespace(context.Context, *CreateNamespaceRequest) (*CreateNamespaceResponse, error) + // Get all namespaces + GetNamespaces(context.Context, *GetNamespacesRequest) (*GetNamespacesResponse, error) + // Get a namespace + GetNamespace(context.Context, *GetNamespaceRequest) (*GetNamespaceResponse, error) + // Update a namespace + UpdateNamespace(context.Context, *UpdateNamespaceRequest) (*UpdateNamespaceResponse, error) + // Rename an existing customer search attribute + RenameCustomSearchAttribute(context.Context, *RenameCustomSearchAttributeRequest) (*RenameCustomSearchAttributeResponse, error) + // Delete a namespace + DeleteNamespace(context.Context, *DeleteNamespaceRequest) (*DeleteNamespaceResponse, error) + // Failover a multi-region namespace + FailoverNamespaceRegion(context.Context, *FailoverNamespaceRegionRequest) (*FailoverNamespaceRegionResponse, error) + // Add a new region to a namespace + AddNamespaceRegion(context.Context, *AddNamespaceRegionRequest) (*AddNamespaceRegionResponse, error) + // Get all regions + GetRegions(context.Context, *GetRegionsRequest) (*GetRegionsResponse, error) + // Get a region + GetRegion(context.Context, *GetRegionRequest) (*GetRegionResponse, error) + // Get all known API keys + GetApiKeys(context.Context, *GetApiKeysRequest) (*GetApiKeysResponse, error) + // Get an API key + GetApiKey(context.Context, *GetApiKeyRequest) (*GetApiKeyResponse, error) + // Create an API key + CreateApiKey(context.Context, *CreateApiKeyRequest) (*CreateApiKeyResponse, error) + // Update an API key + UpdateApiKey(context.Context, *UpdateApiKeyRequest) (*UpdateApiKeyResponse, error) + // Delete an API key + DeleteApiKey(context.Context, *DeleteApiKeyRequest) (*DeleteApiKeyResponse, error) + // Get all user groups + GetUserGroups(context.Context, *GetUserGroupsRequest) (*GetUserGroupsResponse, error) + // Get a user group + GetUserGroup(context.Context, *GetUserGroupRequest) (*GetUserGroupResponse, error) + // Create new a user group + CreateUserGroup(context.Context, *CreateUserGroupRequest) (*CreateUserGroupResponse, error) + // Update a user group + UpdateUserGroup(context.Context, *UpdateUserGroupRequest) (*UpdateUserGroupResponse, error) + // Delete a user group + DeleteUserGroup(context.Context, *DeleteUserGroupRequest) (*DeleteUserGroupResponse, error) + // Set a user group's access to a namespace + SetUserGroupNamespaceAccess(context.Context, *SetUserGroupNamespaceAccessRequest) (*SetUserGroupNamespaceAccessResponse, error) + // Create a service account. + CreateServiceAccount(context.Context, *CreateServiceAccountRequest) (*CreateServiceAccountResponse, error) + // Get a service account. + GetServiceAccount(context.Context, *GetServiceAccountRequest) (*GetServiceAccountResponse, error) + // Get service accounts. + GetServiceAccounts(context.Context, *GetServiceAccountsRequest) (*GetServiceAccountsResponse, error) + // Update a service account. + UpdateServiceAccount(context.Context, *UpdateServiceAccountRequest) (*UpdateServiceAccountResponse, error) + // Delete a service account. + DeleteServiceAccount(context.Context, *DeleteServiceAccountRequest) (*DeleteServiceAccountResponse, error) + mustEmbedUnimplementedCloudServiceServer() +} + +// UnimplementedCloudServiceServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedCloudServiceServer struct{} + +func (UnimplementedCloudServiceServer) GetUsers(context.Context, *GetUsersRequest) (*GetUsersResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetUsers not implemented") +} +func (UnimplementedCloudServiceServer) GetUser(context.Context, *GetUserRequest) (*GetUserResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetUser not implemented") +} +func (UnimplementedCloudServiceServer) CreateUser(context.Context, *CreateUserRequest) (*CreateUserResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateUser not implemented") +} +func (UnimplementedCloudServiceServer) UpdateUser(context.Context, *UpdateUserRequest) (*UpdateUserResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateUser not implemented") +} +func (UnimplementedCloudServiceServer) DeleteUser(context.Context, *DeleteUserRequest) (*DeleteUserResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteUser not implemented") +} +func (UnimplementedCloudServiceServer) SetUserNamespaceAccess(context.Context, *SetUserNamespaceAccessRequest) (*SetUserNamespaceAccessResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SetUserNamespaceAccess not implemented") +} +func (UnimplementedCloudServiceServer) GetAsyncOperation(context.Context, *GetAsyncOperationRequest) (*GetAsyncOperationResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetAsyncOperation not implemented") +} +func (UnimplementedCloudServiceServer) CreateNamespace(context.Context, *CreateNamespaceRequest) (*CreateNamespaceResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateNamespace not implemented") +} +func (UnimplementedCloudServiceServer) GetNamespaces(context.Context, *GetNamespacesRequest) (*GetNamespacesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetNamespaces not implemented") +} +func (UnimplementedCloudServiceServer) GetNamespace(context.Context, *GetNamespaceRequest) (*GetNamespaceResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetNamespace not implemented") +} +func (UnimplementedCloudServiceServer) UpdateNamespace(context.Context, *UpdateNamespaceRequest) (*UpdateNamespaceResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateNamespace not implemented") +} +func (UnimplementedCloudServiceServer) RenameCustomSearchAttribute(context.Context, *RenameCustomSearchAttributeRequest) (*RenameCustomSearchAttributeResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RenameCustomSearchAttribute not implemented") +} +func (UnimplementedCloudServiceServer) DeleteNamespace(context.Context, *DeleteNamespaceRequest) (*DeleteNamespaceResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteNamespace not implemented") +} +func (UnimplementedCloudServiceServer) FailoverNamespaceRegion(context.Context, *FailoverNamespaceRegionRequest) (*FailoverNamespaceRegionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method FailoverNamespaceRegion not implemented") +} +func (UnimplementedCloudServiceServer) AddNamespaceRegion(context.Context, *AddNamespaceRegionRequest) (*AddNamespaceRegionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AddNamespaceRegion not implemented") +} +func (UnimplementedCloudServiceServer) GetRegions(context.Context, *GetRegionsRequest) (*GetRegionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetRegions not implemented") +} +func (UnimplementedCloudServiceServer) GetRegion(context.Context, *GetRegionRequest) (*GetRegionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetRegion not implemented") +} +func (UnimplementedCloudServiceServer) GetApiKeys(context.Context, *GetApiKeysRequest) (*GetApiKeysResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetApiKeys not implemented") +} +func (UnimplementedCloudServiceServer) GetApiKey(context.Context, *GetApiKeyRequest) (*GetApiKeyResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetApiKey not implemented") +} +func (UnimplementedCloudServiceServer) CreateApiKey(context.Context, *CreateApiKeyRequest) (*CreateApiKeyResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateApiKey not implemented") +} +func (UnimplementedCloudServiceServer) UpdateApiKey(context.Context, *UpdateApiKeyRequest) (*UpdateApiKeyResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateApiKey not implemented") +} +func (UnimplementedCloudServiceServer) DeleteApiKey(context.Context, *DeleteApiKeyRequest) (*DeleteApiKeyResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteApiKey not implemented") +} +func (UnimplementedCloudServiceServer) GetUserGroups(context.Context, *GetUserGroupsRequest) (*GetUserGroupsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetUserGroups not implemented") +} +func (UnimplementedCloudServiceServer) GetUserGroup(context.Context, *GetUserGroupRequest) (*GetUserGroupResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetUserGroup not implemented") +} +func (UnimplementedCloudServiceServer) CreateUserGroup(context.Context, *CreateUserGroupRequest) (*CreateUserGroupResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateUserGroup not implemented") +} +func (UnimplementedCloudServiceServer) UpdateUserGroup(context.Context, *UpdateUserGroupRequest) (*UpdateUserGroupResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateUserGroup not implemented") +} +func (UnimplementedCloudServiceServer) DeleteUserGroup(context.Context, *DeleteUserGroupRequest) (*DeleteUserGroupResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteUserGroup not implemented") +} +func (UnimplementedCloudServiceServer) SetUserGroupNamespaceAccess(context.Context, *SetUserGroupNamespaceAccessRequest) (*SetUserGroupNamespaceAccessResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SetUserGroupNamespaceAccess not implemented") +} +func (UnimplementedCloudServiceServer) CreateServiceAccount(context.Context, *CreateServiceAccountRequest) (*CreateServiceAccountResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateServiceAccount not implemented") +} +func (UnimplementedCloudServiceServer) GetServiceAccount(context.Context, *GetServiceAccountRequest) (*GetServiceAccountResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetServiceAccount not implemented") +} +func (UnimplementedCloudServiceServer) GetServiceAccounts(context.Context, *GetServiceAccountsRequest) (*GetServiceAccountsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetServiceAccounts not implemented") +} +func (UnimplementedCloudServiceServer) UpdateServiceAccount(context.Context, *UpdateServiceAccountRequest) (*UpdateServiceAccountResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateServiceAccount not implemented") +} +func (UnimplementedCloudServiceServer) DeleteServiceAccount(context.Context, *DeleteServiceAccountRequest) (*DeleteServiceAccountResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteServiceAccount not implemented") +} +func (UnimplementedCloudServiceServer) mustEmbedUnimplementedCloudServiceServer() {} +func (UnimplementedCloudServiceServer) testEmbeddedByValue() {} + +// UnsafeCloudServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to CloudServiceServer will +// result in compilation errors. +type UnsafeCloudServiceServer interface { + mustEmbedUnimplementedCloudServiceServer() +} + +func RegisterCloudServiceServer(s grpc.ServiceRegistrar, srv CloudServiceServer) { + // If the following call pancis, it indicates UnimplementedCloudServiceServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } + s.RegisterService(&CloudService_ServiceDesc, srv) +} + +func _CloudService_GetUsers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetUsersRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).GetUsers(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_GetUsers_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).GetUsers(ctx, req.(*GetUsersRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_GetUser_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetUserRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).GetUser(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_GetUser_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).GetUser(ctx, req.(*GetUserRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_CreateUser_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateUserRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).CreateUser(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_CreateUser_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).CreateUser(ctx, req.(*CreateUserRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_UpdateUser_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateUserRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).UpdateUser(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_UpdateUser_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).UpdateUser(ctx, req.(*UpdateUserRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_DeleteUser_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteUserRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).DeleteUser(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_DeleteUser_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).DeleteUser(ctx, req.(*DeleteUserRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_SetUserNamespaceAccess_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SetUserNamespaceAccessRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).SetUserNamespaceAccess(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_SetUserNamespaceAccess_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).SetUserNamespaceAccess(ctx, req.(*SetUserNamespaceAccessRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_GetAsyncOperation_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetAsyncOperationRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).GetAsyncOperation(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_GetAsyncOperation_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).GetAsyncOperation(ctx, req.(*GetAsyncOperationRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_CreateNamespace_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateNamespaceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).CreateNamespace(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_CreateNamespace_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).CreateNamespace(ctx, req.(*CreateNamespaceRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_GetNamespaces_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetNamespacesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).GetNamespaces(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_GetNamespaces_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).GetNamespaces(ctx, req.(*GetNamespacesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_GetNamespace_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetNamespaceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).GetNamespace(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_GetNamespace_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).GetNamespace(ctx, req.(*GetNamespaceRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_UpdateNamespace_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateNamespaceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).UpdateNamespace(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_UpdateNamespace_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).UpdateNamespace(ctx, req.(*UpdateNamespaceRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_RenameCustomSearchAttribute_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RenameCustomSearchAttributeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).RenameCustomSearchAttribute(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_RenameCustomSearchAttribute_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).RenameCustomSearchAttribute(ctx, req.(*RenameCustomSearchAttributeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_DeleteNamespace_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteNamespaceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).DeleteNamespace(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_DeleteNamespace_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).DeleteNamespace(ctx, req.(*DeleteNamespaceRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_FailoverNamespaceRegion_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(FailoverNamespaceRegionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).FailoverNamespaceRegion(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_FailoverNamespaceRegion_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).FailoverNamespaceRegion(ctx, req.(*FailoverNamespaceRegionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_AddNamespaceRegion_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AddNamespaceRegionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).AddNamespaceRegion(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_AddNamespaceRegion_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).AddNamespaceRegion(ctx, req.(*AddNamespaceRegionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_GetRegions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetRegionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).GetRegions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_GetRegions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).GetRegions(ctx, req.(*GetRegionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_GetRegion_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetRegionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).GetRegion(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_GetRegion_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).GetRegion(ctx, req.(*GetRegionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_GetApiKeys_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetApiKeysRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).GetApiKeys(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_GetApiKeys_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).GetApiKeys(ctx, req.(*GetApiKeysRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_GetApiKey_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetApiKeyRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).GetApiKey(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_GetApiKey_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).GetApiKey(ctx, req.(*GetApiKeyRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_CreateApiKey_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateApiKeyRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).CreateApiKey(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_CreateApiKey_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).CreateApiKey(ctx, req.(*CreateApiKeyRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_UpdateApiKey_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateApiKeyRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).UpdateApiKey(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_UpdateApiKey_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).UpdateApiKey(ctx, req.(*UpdateApiKeyRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_DeleteApiKey_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteApiKeyRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).DeleteApiKey(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_DeleteApiKey_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).DeleteApiKey(ctx, req.(*DeleteApiKeyRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_GetUserGroups_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetUserGroupsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).GetUserGroups(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_GetUserGroups_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).GetUserGroups(ctx, req.(*GetUserGroupsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_GetUserGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetUserGroupRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).GetUserGroup(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_GetUserGroup_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).GetUserGroup(ctx, req.(*GetUserGroupRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_CreateUserGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateUserGroupRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).CreateUserGroup(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_CreateUserGroup_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).CreateUserGroup(ctx, req.(*CreateUserGroupRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_UpdateUserGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateUserGroupRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).UpdateUserGroup(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_UpdateUserGroup_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).UpdateUserGroup(ctx, req.(*UpdateUserGroupRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_DeleteUserGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteUserGroupRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).DeleteUserGroup(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_DeleteUserGroup_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).DeleteUserGroup(ctx, req.(*DeleteUserGroupRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_SetUserGroupNamespaceAccess_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SetUserGroupNamespaceAccessRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).SetUserGroupNamespaceAccess(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_SetUserGroupNamespaceAccess_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).SetUserGroupNamespaceAccess(ctx, req.(*SetUserGroupNamespaceAccessRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_CreateServiceAccount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateServiceAccountRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).CreateServiceAccount(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_CreateServiceAccount_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).CreateServiceAccount(ctx, req.(*CreateServiceAccountRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_GetServiceAccount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetServiceAccountRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).GetServiceAccount(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_GetServiceAccount_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).GetServiceAccount(ctx, req.(*GetServiceAccountRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_GetServiceAccounts_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetServiceAccountsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).GetServiceAccounts(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_GetServiceAccounts_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).GetServiceAccounts(ctx, req.(*GetServiceAccountsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_UpdateServiceAccount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateServiceAccountRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).UpdateServiceAccount(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_UpdateServiceAccount_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).UpdateServiceAccount(ctx, req.(*UpdateServiceAccountRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _CloudService_DeleteServiceAccount_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteServiceAccountRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(CloudServiceServer).DeleteServiceAccount(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: CloudService_DeleteServiceAccount_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CloudServiceServer).DeleteServiceAccount(ctx, req.(*DeleteServiceAccountRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// CloudService_ServiceDesc is the grpc.ServiceDesc for CloudService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var CloudService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "temporal.api.cloud.cloudservice.v1.CloudService", + HandlerType: (*CloudServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "GetUsers", + Handler: _CloudService_GetUsers_Handler, + }, + { + MethodName: "GetUser", + Handler: _CloudService_GetUser_Handler, + }, + { + MethodName: "CreateUser", + Handler: _CloudService_CreateUser_Handler, + }, + { + MethodName: "UpdateUser", + Handler: _CloudService_UpdateUser_Handler, + }, + { + MethodName: "DeleteUser", + Handler: _CloudService_DeleteUser_Handler, + }, + { + MethodName: "SetUserNamespaceAccess", + Handler: _CloudService_SetUserNamespaceAccess_Handler, + }, + { + MethodName: "GetAsyncOperation", + Handler: _CloudService_GetAsyncOperation_Handler, + }, + { + MethodName: "CreateNamespace", + Handler: _CloudService_CreateNamespace_Handler, + }, + { + MethodName: "GetNamespaces", + Handler: _CloudService_GetNamespaces_Handler, + }, + { + MethodName: "GetNamespace", + Handler: _CloudService_GetNamespace_Handler, + }, + { + MethodName: "UpdateNamespace", + Handler: _CloudService_UpdateNamespace_Handler, + }, + { + MethodName: "RenameCustomSearchAttribute", + Handler: _CloudService_RenameCustomSearchAttribute_Handler, + }, + { + MethodName: "DeleteNamespace", + Handler: _CloudService_DeleteNamespace_Handler, + }, + { + MethodName: "FailoverNamespaceRegion", + Handler: _CloudService_FailoverNamespaceRegion_Handler, + }, + { + MethodName: "AddNamespaceRegion", + Handler: _CloudService_AddNamespaceRegion_Handler, + }, + { + MethodName: "GetRegions", + Handler: _CloudService_GetRegions_Handler, + }, + { + MethodName: "GetRegion", + Handler: _CloudService_GetRegion_Handler, + }, + { + MethodName: "GetApiKeys", + Handler: _CloudService_GetApiKeys_Handler, + }, + { + MethodName: "GetApiKey", + Handler: _CloudService_GetApiKey_Handler, + }, + { + MethodName: "CreateApiKey", + Handler: _CloudService_CreateApiKey_Handler, + }, + { + MethodName: "UpdateApiKey", + Handler: _CloudService_UpdateApiKey_Handler, + }, + { + MethodName: "DeleteApiKey", + Handler: _CloudService_DeleteApiKey_Handler, + }, + { + MethodName: "GetUserGroups", + Handler: _CloudService_GetUserGroups_Handler, + }, + { + MethodName: "GetUserGroup", + Handler: _CloudService_GetUserGroup_Handler, + }, + { + MethodName: "CreateUserGroup", + Handler: _CloudService_CreateUserGroup_Handler, + }, + { + MethodName: "UpdateUserGroup", + Handler: _CloudService_UpdateUserGroup_Handler, + }, + { + MethodName: "DeleteUserGroup", + Handler: _CloudService_DeleteUserGroup_Handler, + }, + { + MethodName: "SetUserGroupNamespaceAccess", + Handler: _CloudService_SetUserGroupNamespaceAccess_Handler, + }, + { + MethodName: "CreateServiceAccount", + Handler: _CloudService_CreateServiceAccount_Handler, + }, + { + MethodName: "GetServiceAccount", + Handler: _CloudService_GetServiceAccount_Handler, + }, + { + MethodName: "GetServiceAccounts", + Handler: _CloudService_GetServiceAccounts_Handler, + }, + { + MethodName: "UpdateServiceAccount", + Handler: _CloudService_UpdateServiceAccount_Handler, + }, + { + MethodName: "DeleteServiceAccount", + Handler: _CloudService_DeleteServiceAccount_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "temporal/api/cloud/cloudservice/v1/service.proto", +} diff --git a/vendor/go.temporal.io/api/cloud/identity/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/cloud/identity/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..ba19f2ec6f5 --- /dev/null +++ b/vendor/go.temporal.io/api/cloud/identity/v1/message.go-helpers.pb.go @@ -0,0 +1,509 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package identity + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type AccountAccess to the protobuf v3 wire format +func (val *AccountAccess) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type AccountAccess from the protobuf v3 wire format +func (val *AccountAccess) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *AccountAccess) Size() int { + return proto.Size(val) +} + +// Equal returns whether two AccountAccess values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *AccountAccess) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *AccountAccess + switch t := that.(type) { + case *AccountAccess: + that1 = t + case AccountAccess: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NamespaceAccess to the protobuf v3 wire format +func (val *NamespaceAccess) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NamespaceAccess from the protobuf v3 wire format +func (val *NamespaceAccess) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NamespaceAccess) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NamespaceAccess values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NamespaceAccess) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NamespaceAccess + switch t := that.(type) { + case *NamespaceAccess: + that1 = t + case NamespaceAccess: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Access to the protobuf v3 wire format +func (val *Access) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Access from the protobuf v3 wire format +func (val *Access) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Access) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Access values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Access) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Access + switch t := that.(type) { + case *Access: + that1 = t + case Access: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UserSpec to the protobuf v3 wire format +func (val *UserSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UserSpec from the protobuf v3 wire format +func (val *UserSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UserSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UserSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UserSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UserSpec + switch t := that.(type) { + case *UserSpec: + that1 = t + case UserSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Invitation to the protobuf v3 wire format +func (val *Invitation) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Invitation from the protobuf v3 wire format +func (val *Invitation) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Invitation) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Invitation values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Invitation) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Invitation + switch t := that.(type) { + case *Invitation: + that1 = t + case Invitation: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type User to the protobuf v3 wire format +func (val *User) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type User from the protobuf v3 wire format +func (val *User) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *User) Size() int { + return proto.Size(val) +} + +// Equal returns whether two User values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *User) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *User + switch t := that.(type) { + case *User: + that1 = t + case User: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GoogleGroupSpec to the protobuf v3 wire format +func (val *GoogleGroupSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GoogleGroupSpec from the protobuf v3 wire format +func (val *GoogleGroupSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GoogleGroupSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GoogleGroupSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GoogleGroupSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GoogleGroupSpec + switch t := that.(type) { + case *GoogleGroupSpec: + that1 = t + case GoogleGroupSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UserGroupSpec to the protobuf v3 wire format +func (val *UserGroupSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UserGroupSpec from the protobuf v3 wire format +func (val *UserGroupSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UserGroupSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UserGroupSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UserGroupSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UserGroupSpec + switch t := that.(type) { + case *UserGroupSpec: + that1 = t + case UserGroupSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UserGroup to the protobuf v3 wire format +func (val *UserGroup) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UserGroup from the protobuf v3 wire format +func (val *UserGroup) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UserGroup) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UserGroup values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UserGroup) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UserGroup + switch t := that.(type) { + case *UserGroup: + that1 = t + case UserGroup: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ServiceAccount to the protobuf v3 wire format +func (val *ServiceAccount) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ServiceAccount from the protobuf v3 wire format +func (val *ServiceAccount) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ServiceAccount) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ServiceAccount values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ServiceAccount) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ServiceAccount + switch t := that.(type) { + case *ServiceAccount: + that1 = t + case ServiceAccount: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ServiceAccountSpec to the protobuf v3 wire format +func (val *ServiceAccountSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ServiceAccountSpec from the protobuf v3 wire format +func (val *ServiceAccountSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ServiceAccountSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ServiceAccountSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ServiceAccountSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ServiceAccountSpec + switch t := that.(type) { + case *ServiceAccountSpec: + that1 = t + case ServiceAccountSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ApiKey to the protobuf v3 wire format +func (val *ApiKey) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ApiKey from the protobuf v3 wire format +func (val *ApiKey) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ApiKey) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ApiKey values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ApiKey) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ApiKey + switch t := that.(type) { + case *ApiKey: + that1 = t + case ApiKey: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ApiKeySpec to the protobuf v3 wire format +func (val *ApiKeySpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ApiKeySpec from the protobuf v3 wire format +func (val *ApiKeySpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ApiKeySpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ApiKeySpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ApiKeySpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ApiKeySpec + switch t := that.(type) { + case *ApiKeySpec: + that1 = t + case ApiKeySpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/cloud/identity/v1/message.pb.go b/vendor/go.temporal.io/api/cloud/identity/v1/message.pb.go new file mode 100644 index 00000000000..205c34d984d --- /dev/null +++ b/vendor/go.temporal.io/api/cloud/identity/v1/message.pb.go @@ -0,0 +1,1470 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/cloud/identity/v1/message.proto + +package identity + +import ( + reflect "reflect" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type AccountAccess struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The role on the account, should be one of [admin, developer, read] + // admin - gives full access the account, including users and namespaces + // developer - gives access to create namespaces on the account + // read - gives read only access to the account + Role string `protobuf:"bytes,1,opt,name=role,proto3" json:"role,omitempty"` +} + +func (x *AccountAccess) Reset() { + *x = AccountAccess{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AccountAccess) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AccountAccess) ProtoMessage() {} + +func (x *AccountAccess) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AccountAccess.ProtoReflect.Descriptor instead. +func (*AccountAccess) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *AccountAccess) GetRole() string { + if x != nil { + return x.Role + } + return "" +} + +type NamespaceAccess struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The permission to the namespace, should be one of [admin, write, read] + // admin - gives full access to the namespace, including assigning namespace access to other users + // write - gives write access to the namespace configuration and workflows within the namespace + // read - gives read only access to the namespace configuration and workflows within the namespace + Permission string `protobuf:"bytes,1,opt,name=permission,proto3" json:"permission,omitempty"` +} + +func (x *NamespaceAccess) Reset() { + *x = NamespaceAccess{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NamespaceAccess) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NamespaceAccess) ProtoMessage() {} + +func (x *NamespaceAccess) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NamespaceAccess.ProtoReflect.Descriptor instead. +func (*NamespaceAccess) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *NamespaceAccess) GetPermission() string { + if x != nil { + return x.Permission + } + return "" +} + +type Access struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The account access + AccountAccess *AccountAccess `protobuf:"bytes,1,opt,name=account_access,json=accountAccess,proto3" json:"account_access,omitempty"` + // The map of namespace accesses + // The key is the namespace name and the value is the access to the namespace + NamespaceAccesses map[string]*NamespaceAccess `protobuf:"bytes,2,rep,name=namespace_accesses,json=namespaceAccesses,proto3" json:"namespace_accesses,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *Access) Reset() { + *x = Access{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Access) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Access) ProtoMessage() {} + +func (x *Access) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Access.ProtoReflect.Descriptor instead. +func (*Access) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *Access) GetAccountAccess() *AccountAccess { + if x != nil { + return x.AccountAccess + } + return nil +} + +func (x *Access) GetNamespaceAccesses() map[string]*NamespaceAccess { + if x != nil { + return x.NamespaceAccesses + } + return nil +} + +type UserSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The email address associated to the user + Email string `protobuf:"bytes,1,opt,name=email,proto3" json:"email,omitempty"` + // The access to assigned to the user + Access *Access `protobuf:"bytes,2,opt,name=access,proto3" json:"access,omitempty"` +} + +func (x *UserSpec) Reset() { + *x = UserSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UserSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UserSpec) ProtoMessage() {} + +func (x *UserSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UserSpec.ProtoReflect.Descriptor instead. +func (*UserSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *UserSpec) GetEmail() string { + if x != nil { + return x.Email + } + return "" +} + +func (x *UserSpec) GetAccess() *Access { + if x != nil { + return x.Access + } + return nil +} + +type Invitation struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The date and time when the user was created + CreatedTime *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"` + // The date and time when the invitation expires or has expired + ExpiredTime *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=expired_time,json=expiredTime,proto3" json:"expired_time,omitempty"` +} + +func (x *Invitation) Reset() { + *x = Invitation{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Invitation) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Invitation) ProtoMessage() {} + +func (x *Invitation) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Invitation.ProtoReflect.Descriptor instead. +func (*Invitation) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *Invitation) GetCreatedTime() *timestamppb.Timestamp { + if x != nil { + return x.CreatedTime + } + return nil +} + +func (x *Invitation) GetExpiredTime() *timestamppb.Timestamp { + if x != nil { + return x.ExpiredTime + } + return nil +} + +type User struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the user + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + // The current version of the user specification + // The next update operation will have to include this version + ResourceVersion string `protobuf:"bytes,2,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The user specification + Spec *UserSpec `protobuf:"bytes,3,opt,name=spec,proto3" json:"spec,omitempty"` + // The current state of the user + State string `protobuf:"bytes,4,opt,name=state,proto3" json:"state,omitempty"` + // The id of the async operation that is creating/updating/deleting the user, if any + AsyncOperationId string `protobuf:"bytes,5,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` + // The details of the open invitation sent to the user, if any + Invitation *Invitation `protobuf:"bytes,6,opt,name=invitation,proto3" json:"invitation,omitempty"` + // The date and time when the user was created + CreatedTime *timestamppb.Timestamp `protobuf:"bytes,7,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"` + // The date and time when the user was last modified + // Will not be set if the user has never been modified + LastModifiedTime *timestamppb.Timestamp `protobuf:"bytes,8,opt,name=last_modified_time,json=lastModifiedTime,proto3" json:"last_modified_time,omitempty"` +} + +func (x *User) Reset() { + *x = User{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *User) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*User) ProtoMessage() {} + +func (x *User) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use User.ProtoReflect.Descriptor instead. +func (*User) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP(), []int{5} +} + +func (x *User) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *User) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *User) GetSpec() *UserSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *User) GetState() string { + if x != nil { + return x.State + } + return "" +} + +func (x *User) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +func (x *User) GetInvitation() *Invitation { + if x != nil { + return x.Invitation + } + return nil +} + +func (x *User) GetCreatedTime() *timestamppb.Timestamp { + if x != nil { + return x.CreatedTime + } + return nil +} + +func (x *User) GetLastModifiedTime() *timestamppb.Timestamp { + if x != nil { + return x.LastModifiedTime + } + return nil +} + +type GoogleGroupSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The email address of the Google group. + // The email address is immutable. Once set during creation, it cannot be changed. + EmailAddress string `protobuf:"bytes,1,opt,name=email_address,json=emailAddress,proto3" json:"email_address,omitempty"` +} + +func (x *GoogleGroupSpec) Reset() { + *x = GoogleGroupSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GoogleGroupSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GoogleGroupSpec) ProtoMessage() {} + +func (x *GoogleGroupSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GoogleGroupSpec.ProtoReflect.Descriptor instead. +func (*GoogleGroupSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP(), []int{6} +} + +func (x *GoogleGroupSpec) GetEmailAddress() string { + if x != nil { + return x.EmailAddress + } + return "" +} + +type UserGroupSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The display name of the group. + DisplayName string `protobuf:"bytes,1,opt,name=display_name,json=displayName,proto3" json:"display_name,omitempty"` + // The access assigned to the group. + Access *Access `protobuf:"bytes,2,opt,name=access,proto3" json:"access,omitempty"` + // The specification of the google group that this group is associated with. + // For now only google groups are supported, and this field is required. + GoogleGroup *GoogleGroupSpec `protobuf:"bytes,3,opt,name=google_group,json=googleGroup,proto3" json:"google_group,omitempty"` +} + +func (x *UserGroupSpec) Reset() { + *x = UserGroupSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UserGroupSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UserGroupSpec) ProtoMessage() {} + +func (x *UserGroupSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UserGroupSpec.ProtoReflect.Descriptor instead. +func (*UserGroupSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP(), []int{7} +} + +func (x *UserGroupSpec) GetDisplayName() string { + if x != nil { + return x.DisplayName + } + return "" +} + +func (x *UserGroupSpec) GetAccess() *Access { + if x != nil { + return x.Access + } + return nil +} + +func (x *UserGroupSpec) GetGoogleGroup() *GoogleGroupSpec { + if x != nil { + return x.GoogleGroup + } + return nil +} + +type UserGroup struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the group + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + // The current version of the group specification + // The next update operation will have to include this version + ResourceVersion string `protobuf:"bytes,2,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The group specification + Spec *UserGroupSpec `protobuf:"bytes,3,opt,name=spec,proto3" json:"spec,omitempty"` + // The current state of the group + State string `protobuf:"bytes,4,opt,name=state,proto3" json:"state,omitempty"` + // The id of the async operation that is creating/updating/deleting the group, if any + AsyncOperationId string `protobuf:"bytes,5,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` + // The date and time when the group was created + CreatedTime *timestamppb.Timestamp `protobuf:"bytes,6,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"` + // The date and time when the group was last modified + // Will not be set if the group has never been modified + LastModifiedTime *timestamppb.Timestamp `protobuf:"bytes,7,opt,name=last_modified_time,json=lastModifiedTime,proto3" json:"last_modified_time,omitempty"` +} + +func (x *UserGroup) Reset() { + *x = UserGroup{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UserGroup) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UserGroup) ProtoMessage() {} + +func (x *UserGroup) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UserGroup.ProtoReflect.Descriptor instead. +func (*UserGroup) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP(), []int{8} +} + +func (x *UserGroup) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *UserGroup) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *UserGroup) GetSpec() *UserGroupSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *UserGroup) GetState() string { + if x != nil { + return x.State + } + return "" +} + +func (x *UserGroup) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +func (x *UserGroup) GetCreatedTime() *timestamppb.Timestamp { + if x != nil { + return x.CreatedTime + } + return nil +} + +func (x *UserGroup) GetLastModifiedTime() *timestamppb.Timestamp { + if x != nil { + return x.LastModifiedTime + } + return nil +} + +type ServiceAccount struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the service account. + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + // The current version of the service account specification. + // The next update operation will have to include this version. + ResourceVersion string `protobuf:"bytes,2,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The service account specification. + Spec *ServiceAccountSpec `protobuf:"bytes,3,opt,name=spec,proto3" json:"spec,omitempty"` + // The current state of the service account. + // Possible values: activating, activationfailed, active, updating, updatefailed, deleting, deletefailed, deleted, suspending, suspendfailed, suspended. + // For any failed state, reach out to Temporal Cloud support for remediation. + State string `protobuf:"bytes,4,opt,name=state,proto3" json:"state,omitempty"` + // The id of the async operation that is creating/updating/deleting the service account, if any. + AsyncOperationId string `protobuf:"bytes,5,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` + // The date and time when the service account was created. + CreatedTime *timestamppb.Timestamp `protobuf:"bytes,6,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"` + // The date and time when the service account was last modified + // Will not be set if the service account has never been modified. + LastModifiedTime *timestamppb.Timestamp `protobuf:"bytes,7,opt,name=last_modified_time,json=lastModifiedTime,proto3" json:"last_modified_time,omitempty"` +} + +func (x *ServiceAccount) Reset() { + *x = ServiceAccount{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ServiceAccount) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ServiceAccount) ProtoMessage() {} + +func (x *ServiceAccount) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ServiceAccount.ProtoReflect.Descriptor instead. +func (*ServiceAccount) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP(), []int{9} +} + +func (x *ServiceAccount) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ServiceAccount) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *ServiceAccount) GetSpec() *ServiceAccountSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *ServiceAccount) GetState() string { + if x != nil { + return x.State + } + return "" +} + +func (x *ServiceAccount) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +func (x *ServiceAccount) GetCreatedTime() *timestamppb.Timestamp { + if x != nil { + return x.CreatedTime + } + return nil +} + +func (x *ServiceAccount) GetLastModifiedTime() *timestamppb.Timestamp { + if x != nil { + return x.LastModifiedTime + } + return nil +} + +type ServiceAccountSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The name associated with the service account. + // The name is mutable, but must be unique across all your active service accounts. + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // The access assigned to the service account. + // The access is mutable. + Access *Access `protobuf:"bytes,2,opt,name=access,proto3" json:"access,omitempty"` + // The description associated with the service account - optional. + // The description is mutable. + Description string `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"` +} + +func (x *ServiceAccountSpec) Reset() { + *x = ServiceAccountSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ServiceAccountSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ServiceAccountSpec) ProtoMessage() {} + +func (x *ServiceAccountSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ServiceAccountSpec.ProtoReflect.Descriptor instead. +func (*ServiceAccountSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP(), []int{10} +} + +func (x *ServiceAccountSpec) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *ServiceAccountSpec) GetAccess() *Access { + if x != nil { + return x.Access + } + return nil +} + +func (x *ServiceAccountSpec) GetDescription() string { + if x != nil { + return x.Description + } + return "" +} + +type ApiKey struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the API Key. + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + // The current version of the API key specification. + // The next update operation will have to include this version. + ResourceVersion string `protobuf:"bytes,2,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The API key specification. + Spec *ApiKeySpec `protobuf:"bytes,3,opt,name=spec,proto3" json:"spec,omitempty"` + // The current state of the API key. + // Possible values: activating, activationfailed, active, updating, updatefailed, deleting, deletefailed, deleted, suspending, suspendfailed, suspended. + // For any failed state, reach out to Temporal Cloud support for remediation. + State string `protobuf:"bytes,4,opt,name=state,proto3" json:"state,omitempty"` + // The id of the async operation that is creating/updating/deleting the API key, if any. + AsyncOperationId string `protobuf:"bytes,5,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` + // The date and time when the API key was created. + CreatedTime *timestamppb.Timestamp `protobuf:"bytes,6,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"` + // The date and time when the API key was last modified. + // Will not be set if the API key has never been modified. + LastModifiedTime *timestamppb.Timestamp `protobuf:"bytes,7,opt,name=last_modified_time,json=lastModifiedTime,proto3" json:"last_modified_time,omitempty"` +} + +func (x *ApiKey) Reset() { + *x = ApiKey{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ApiKey) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ApiKey) ProtoMessage() {} + +func (x *ApiKey) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ApiKey.ProtoReflect.Descriptor instead. +func (*ApiKey) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP(), []int{11} +} + +func (x *ApiKey) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ApiKey) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *ApiKey) GetSpec() *ApiKeySpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *ApiKey) GetState() string { + if x != nil { + return x.State + } + return "" +} + +func (x *ApiKey) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +func (x *ApiKey) GetCreatedTime() *timestamppb.Timestamp { + if x != nil { + return x.CreatedTime + } + return nil +} + +func (x *ApiKey) GetLastModifiedTime() *timestamppb.Timestamp { + if x != nil { + return x.LastModifiedTime + } + return nil +} + +type ApiKeySpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the owner to create the API key for. + // The owner id is immutable. Once set during creation, it cannot be changed. + // The owner id is the id of the user when the owner type is 'user'. + // The owner id is the id of the service account when the owner type is 'service-account'. + OwnerId string `protobuf:"bytes,1,opt,name=owner_id,json=ownerId,proto3" json:"owner_id,omitempty"` + // The type of the owner to create the API key for. + // The owner type is immutable. Once set during creation, it cannot be changed. + // Possible values: user, service-account. + OwnerType string `protobuf:"bytes,2,opt,name=owner_type,json=ownerType,proto3" json:"owner_type,omitempty"` + // The display name of the API key. + DisplayName string `protobuf:"bytes,3,opt,name=display_name,json=displayName,proto3" json:"display_name,omitempty"` + // The description of the API key. + Description string `protobuf:"bytes,4,opt,name=description,proto3" json:"description,omitempty"` + // The expiry time of the API key. + ExpiryTime *timestamppb.Timestamp `protobuf:"bytes,5,opt,name=expiry_time,json=expiryTime,proto3" json:"expiry_time,omitempty"` + // True if the API key is disabled. + Disabled bool `protobuf:"varint,6,opt,name=disabled,proto3" json:"disabled,omitempty"` +} + +func (x *ApiKeySpec) Reset() { + *x = ApiKeySpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ApiKeySpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ApiKeySpec) ProtoMessage() {} + +func (x *ApiKeySpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_identity_v1_message_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ApiKeySpec.ProtoReflect.Descriptor instead. +func (*ApiKeySpec) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP(), []int{12} +} + +func (x *ApiKeySpec) GetOwnerId() string { + if x != nil { + return x.OwnerId + } + return "" +} + +func (x *ApiKeySpec) GetOwnerType() string { + if x != nil { + return x.OwnerType + } + return "" +} + +func (x *ApiKeySpec) GetDisplayName() string { + if x != nil { + return x.DisplayName + } + return "" +} + +func (x *ApiKeySpec) GetDescription() string { + if x != nil { + return x.Description + } + return "" +} + +func (x *ApiKeySpec) GetExpiryTime() *timestamppb.Timestamp { + if x != nil { + return x.ExpiryTime + } + return nil +} + +func (x *ApiKeySpec) GetDisabled() bool { + if x != nil { + return x.Disabled + } + return false +} + +var File_temporal_api_cloud_identity_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_cloud_identity_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x2c, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x1e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x27, 0x0a, 0x0d, 0x41, 0x63, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x16, 0x0a, 0x04, 0x72, 0x6f, 0x6c, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x72, 0x6f, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x35, + 0x0a, 0x0f, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, + 0x22, 0x0a, 0x0a, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0a, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x22, 0xd3, 0x02, 0x0a, 0x06, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x58, 0x0a, 0x0e, 0x61, + 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, + 0x31, 0x2e, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x0d, + 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x70, 0x0a, 0x12, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, + 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x2e, + 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x65, 0x73, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x11, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, + 0x63, 0x63, 0x65, 0x73, 0x73, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x7d, 0x0a, 0x16, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x65, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x49, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, + 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x63, 0x63, + 0x65, 0x73, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x68, 0x0a, 0x08, 0x55, 0x73, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x12, 0x18, 0x0a, 0x05, + 0x65, 0x6d, 0x61, 0x69, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x6d, 0x61, 0x69, + 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x42, 0x0a, 0x06, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, + 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x06, 0x61, 0x63, 0x63, 0x65, 0x73, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x92, 0x01, 0x0a, 0x0a, 0x49, 0x6e, 0x76, 0x69, 0x74, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x41, 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x41, 0x0a, 0x0c, 0x65, 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x52, 0x0b, 0x65, 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x22, 0xb8, 0x03, 0x0a, 0x04, 0x55, 0x73, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x40, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, + 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, + 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, + 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4e, 0x0a, 0x0a, 0x69, + 0x6e, 0x76, 0x69, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, + 0x76, 0x69, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x69, 0x6e, 0x76, 0x69, 0x74, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, + 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4c, 0x0a, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x6f, + 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x10, 0x6c, 0x61, 0x73, 0x74, 0x4d, + 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x3a, + 0x0a, 0x0f, 0x47, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, + 0x12, 0x27, 0x0a, 0x0d, 0x65, 0x6d, 0x61, 0x69, 0x6c, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x6d, 0x61, 0x69, 0x6c, 0x41, 0x64, 0x64, 0x72, + 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd2, 0x01, 0x0a, 0x0d, 0x55, 0x73, 0x65, 0x72, 0x47, + 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x12, 0x25, 0x0a, 0x0c, 0x64, 0x69, 0x73, 0x70, 0x6c, + 0x61, 0x79, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x69, + 0x73, 0x70, 0x6c, 0x61, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x42, 0x0a, 0x06, 0x61, + 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, + 0x52, 0x06, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x56, 0x0a, 0x0c, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, + 0x47, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0b, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x02, 0x68, 0x00, 0x22, 0xf2, + 0x02, 0x0a, 0x09, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x12, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, + 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x45, 0x0a, 0x04, 0x73, 0x70, 0x65, + 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, + 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, + 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x4c, 0x0a, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, + 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x10, 0x6c, 0x61, 0x73, 0x74, 0x4d, 0x6f, 0x64, 0x69, 0x66, + 0x69, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xfc, 0x02, 0x0a, 0x0e, 0x53, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x12, 0x0a, 0x02, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x04, 0x73, + 0x70, 0x65, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, + 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, + 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x0c, 0x63, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4c, 0x0a, 0x12, 0x6c, 0x61, 0x73, 0x74, + 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x10, 0x6c, 0x61, + 0x73, 0x74, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x96, 0x01, 0x0a, 0x12, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x53, 0x70, 0x65, 0x63, 0x12, 0x16, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x42, 0x0a, + 0x06, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, + 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x63, 0x65, + 0x73, 0x73, 0x52, 0x06, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x24, 0x0a, 0x0b, + 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, + 0x22, 0xec, 0x02, 0x0a, 0x06, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x12, 0x12, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, + 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x42, 0x0a, 0x04, 0x73, 0x70, 0x65, + 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x4b, 0x65, 0x79, 0x53, 0x70, 0x65, 0x63, 0x52, + 0x04, 0x73, 0x70, 0x65, 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, + 0x6e, 0x63, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x41, 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, + 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x4c, 0x0a, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x5f, 0x74, + 0x69, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x52, 0x10, 0x6c, 0x61, 0x73, 0x74, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, + 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xfc, 0x01, 0x0a, 0x0a, 0x41, 0x70, 0x69, 0x4b, + 0x65, 0x79, 0x53, 0x70, 0x65, 0x63, 0x12, 0x1d, 0x0a, 0x08, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x5f, 0x74, 0x79, 0x70, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x54, 0x79, 0x70, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, + 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x24, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, 0x0a, 0x0b, 0x65, + 0x78, 0x70, 0x69, 0x72, 0x79, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0a, 0x65, 0x78, 0x70, 0x69, 0x72, + 0x79, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x64, 0x69, 0x73, 0x61, + 0x62, 0x6c, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x64, 0x69, 0x73, 0x61, 0x62, + 0x6c, 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x42, 0xac, 0x01, 0x0a, 0x21, 0x69, 0x6f, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x69, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x2d, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x2f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x2f, 0x76, 0x31, 0x3b, 0x69, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x74, 0x79, 0xaa, 0x02, 0x20, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, + 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x49, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x74, 0x79, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x24, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x3a, 0x3a, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, +} + +var ( + file_temporal_api_cloud_identity_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_cloud_identity_v1_message_proto_rawDescData = file_temporal_api_cloud_identity_v1_message_proto_rawDesc +) + +func file_temporal_api_cloud_identity_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_cloud_identity_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_cloud_identity_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_cloud_identity_v1_message_proto_rawDescData) + }) + return file_temporal_api_cloud_identity_v1_message_proto_rawDescData +} + +var file_temporal_api_cloud_identity_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 14) +var file_temporal_api_cloud_identity_v1_message_proto_goTypes = []any{ + (*AccountAccess)(nil), // 0: temporal.api.cloud.identity.v1.AccountAccess + (*NamespaceAccess)(nil), // 1: temporal.api.cloud.identity.v1.NamespaceAccess + (*Access)(nil), // 2: temporal.api.cloud.identity.v1.Access + (*UserSpec)(nil), // 3: temporal.api.cloud.identity.v1.UserSpec + (*Invitation)(nil), // 4: temporal.api.cloud.identity.v1.Invitation + (*User)(nil), // 5: temporal.api.cloud.identity.v1.User + (*GoogleGroupSpec)(nil), // 6: temporal.api.cloud.identity.v1.GoogleGroupSpec + (*UserGroupSpec)(nil), // 7: temporal.api.cloud.identity.v1.UserGroupSpec + (*UserGroup)(nil), // 8: temporal.api.cloud.identity.v1.UserGroup + (*ServiceAccount)(nil), // 9: temporal.api.cloud.identity.v1.ServiceAccount + (*ServiceAccountSpec)(nil), // 10: temporal.api.cloud.identity.v1.ServiceAccountSpec + (*ApiKey)(nil), // 11: temporal.api.cloud.identity.v1.ApiKey + (*ApiKeySpec)(nil), // 12: temporal.api.cloud.identity.v1.ApiKeySpec + nil, // 13: temporal.api.cloud.identity.v1.Access.NamespaceAccessesEntry + (*timestamppb.Timestamp)(nil), // 14: google.protobuf.Timestamp +} +var file_temporal_api_cloud_identity_v1_message_proto_depIdxs = []int32{ + 0, // 0: temporal.api.cloud.identity.v1.Access.account_access:type_name -> temporal.api.cloud.identity.v1.AccountAccess + 13, // 1: temporal.api.cloud.identity.v1.Access.namespace_accesses:type_name -> temporal.api.cloud.identity.v1.Access.NamespaceAccessesEntry + 2, // 2: temporal.api.cloud.identity.v1.UserSpec.access:type_name -> temporal.api.cloud.identity.v1.Access + 14, // 3: temporal.api.cloud.identity.v1.Invitation.created_time:type_name -> google.protobuf.Timestamp + 14, // 4: temporal.api.cloud.identity.v1.Invitation.expired_time:type_name -> google.protobuf.Timestamp + 3, // 5: temporal.api.cloud.identity.v1.User.spec:type_name -> temporal.api.cloud.identity.v1.UserSpec + 4, // 6: temporal.api.cloud.identity.v1.User.invitation:type_name -> temporal.api.cloud.identity.v1.Invitation + 14, // 7: temporal.api.cloud.identity.v1.User.created_time:type_name -> google.protobuf.Timestamp + 14, // 8: temporal.api.cloud.identity.v1.User.last_modified_time:type_name -> google.protobuf.Timestamp + 2, // 9: temporal.api.cloud.identity.v1.UserGroupSpec.access:type_name -> temporal.api.cloud.identity.v1.Access + 6, // 10: temporal.api.cloud.identity.v1.UserGroupSpec.google_group:type_name -> temporal.api.cloud.identity.v1.GoogleGroupSpec + 7, // 11: temporal.api.cloud.identity.v1.UserGroup.spec:type_name -> temporal.api.cloud.identity.v1.UserGroupSpec + 14, // 12: temporal.api.cloud.identity.v1.UserGroup.created_time:type_name -> google.protobuf.Timestamp + 14, // 13: temporal.api.cloud.identity.v1.UserGroup.last_modified_time:type_name -> google.protobuf.Timestamp + 10, // 14: temporal.api.cloud.identity.v1.ServiceAccount.spec:type_name -> temporal.api.cloud.identity.v1.ServiceAccountSpec + 14, // 15: temporal.api.cloud.identity.v1.ServiceAccount.created_time:type_name -> google.protobuf.Timestamp + 14, // 16: temporal.api.cloud.identity.v1.ServiceAccount.last_modified_time:type_name -> google.protobuf.Timestamp + 2, // 17: temporal.api.cloud.identity.v1.ServiceAccountSpec.access:type_name -> temporal.api.cloud.identity.v1.Access + 12, // 18: temporal.api.cloud.identity.v1.ApiKey.spec:type_name -> temporal.api.cloud.identity.v1.ApiKeySpec + 14, // 19: temporal.api.cloud.identity.v1.ApiKey.created_time:type_name -> google.protobuf.Timestamp + 14, // 20: temporal.api.cloud.identity.v1.ApiKey.last_modified_time:type_name -> google.protobuf.Timestamp + 14, // 21: temporal.api.cloud.identity.v1.ApiKeySpec.expiry_time:type_name -> google.protobuf.Timestamp + 1, // 22: temporal.api.cloud.identity.v1.Access.NamespaceAccessesEntry.value:type_name -> temporal.api.cloud.identity.v1.NamespaceAccess + 23, // [23:23] is the sub-list for method output_type + 23, // [23:23] is the sub-list for method input_type + 23, // [23:23] is the sub-list for extension type_name + 23, // [23:23] is the sub-list for extension extendee + 0, // [0:23] is the sub-list for field type_name +} + +func init() { file_temporal_api_cloud_identity_v1_message_proto_init() } +func file_temporal_api_cloud_identity_v1_message_proto_init() { + if File_temporal_api_cloud_identity_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_cloud_identity_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*AccountAccess); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_identity_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*NamespaceAccess); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_identity_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*Access); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_identity_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*UserSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_identity_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*Invitation); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_identity_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*User); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_identity_v1_message_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*GoogleGroupSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_identity_v1_message_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*UserGroupSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_identity_v1_message_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*UserGroup); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_identity_v1_message_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*ServiceAccount); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_identity_v1_message_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*ServiceAccountSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_identity_v1_message_proto_msgTypes[11].Exporter = func(v any, i int) any { + switch v := v.(*ApiKey); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_identity_v1_message_proto_msgTypes[12].Exporter = func(v any, i int) any { + switch v := v.(*ApiKeySpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_cloud_identity_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 14, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_cloud_identity_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_cloud_identity_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_cloud_identity_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_cloud_identity_v1_message_proto = out.File + file_temporal_api_cloud_identity_v1_message_proto_rawDesc = nil + file_temporal_api_cloud_identity_v1_message_proto_goTypes = nil + file_temporal_api_cloud_identity_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/cloud/namespace/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/cloud/namespace/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..1a4a11d1229 --- /dev/null +++ b/vendor/go.temporal.io/api/cloud/namespace/v1/message.go-helpers.pb.go @@ -0,0 +1,435 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package namespace + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type CertificateFilterSpec to the protobuf v3 wire format +func (val *CertificateFilterSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CertificateFilterSpec from the protobuf v3 wire format +func (val *CertificateFilterSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CertificateFilterSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CertificateFilterSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CertificateFilterSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CertificateFilterSpec + switch t := that.(type) { + case *CertificateFilterSpec: + that1 = t + case CertificateFilterSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type MtlsAuthSpec to the protobuf v3 wire format +func (val *MtlsAuthSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type MtlsAuthSpec from the protobuf v3 wire format +func (val *MtlsAuthSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *MtlsAuthSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two MtlsAuthSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *MtlsAuthSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *MtlsAuthSpec + switch t := that.(type) { + case *MtlsAuthSpec: + that1 = t + case MtlsAuthSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ApiKeyAuthSpec to the protobuf v3 wire format +func (val *ApiKeyAuthSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ApiKeyAuthSpec from the protobuf v3 wire format +func (val *ApiKeyAuthSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ApiKeyAuthSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ApiKeyAuthSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ApiKeyAuthSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ApiKeyAuthSpec + switch t := that.(type) { + case *ApiKeyAuthSpec: + that1 = t + case ApiKeyAuthSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CodecServerSpec to the protobuf v3 wire format +func (val *CodecServerSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CodecServerSpec from the protobuf v3 wire format +func (val *CodecServerSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CodecServerSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CodecServerSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CodecServerSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CodecServerSpec + switch t := that.(type) { + case *CodecServerSpec: + that1 = t + case CodecServerSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NamespaceSpec to the protobuf v3 wire format +func (val *NamespaceSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NamespaceSpec from the protobuf v3 wire format +func (val *NamespaceSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NamespaceSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NamespaceSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NamespaceSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NamespaceSpec + switch t := that.(type) { + case *NamespaceSpec: + that1 = t + case NamespaceSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Endpoints to the protobuf v3 wire format +func (val *Endpoints) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Endpoints from the protobuf v3 wire format +func (val *Endpoints) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Endpoints) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Endpoints values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Endpoints) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Endpoints + switch t := that.(type) { + case *Endpoints: + that1 = t + case Endpoints: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Limits to the protobuf v3 wire format +func (val *Limits) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Limits from the protobuf v3 wire format +func (val *Limits) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Limits) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Limits values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Limits) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Limits + switch t := that.(type) { + case *Limits: + that1 = t + case Limits: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type AWSPrivateLinkInfo to the protobuf v3 wire format +func (val *AWSPrivateLinkInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type AWSPrivateLinkInfo from the protobuf v3 wire format +func (val *AWSPrivateLinkInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *AWSPrivateLinkInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two AWSPrivateLinkInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *AWSPrivateLinkInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *AWSPrivateLinkInfo + switch t := that.(type) { + case *AWSPrivateLinkInfo: + that1 = t + case AWSPrivateLinkInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PrivateConnectivity to the protobuf v3 wire format +func (val *PrivateConnectivity) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PrivateConnectivity from the protobuf v3 wire format +func (val *PrivateConnectivity) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PrivateConnectivity) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PrivateConnectivity values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PrivateConnectivity) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PrivateConnectivity + switch t := that.(type) { + case *PrivateConnectivity: + that1 = t + case PrivateConnectivity: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Namespace to the protobuf v3 wire format +func (val *Namespace) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Namespace from the protobuf v3 wire format +func (val *Namespace) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Namespace) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Namespace values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Namespace) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Namespace + switch t := that.(type) { + case *Namespace: + that1 = t + case Namespace: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NamespaceRegionStatus to the protobuf v3 wire format +func (val *NamespaceRegionStatus) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NamespaceRegionStatus from the protobuf v3 wire format +func (val *NamespaceRegionStatus) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NamespaceRegionStatus) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NamespaceRegionStatus values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NamespaceRegionStatus) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NamespaceRegionStatus + switch t := that.(type) { + case *NamespaceRegionStatus: + that1 = t + case NamespaceRegionStatus: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/cloud/namespace/v1/message.pb.go b/vendor/go.temporal.io/api/cloud/namespace/v1/message.pb.go new file mode 100644 index 00000000000..44993a4639c --- /dev/null +++ b/vendor/go.temporal.io/api/cloud/namespace/v1/message.pb.go @@ -0,0 +1,1268 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/cloud/namespace/v1/message.proto + +package namespace + +import ( + reflect "reflect" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type CertificateFilterSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The common_name in the certificate. + // Optional, default is empty. + CommonName string `protobuf:"bytes,1,opt,name=common_name,json=commonName,proto3" json:"common_name,omitempty"` + // The organization in the certificate. + // Optional, default is empty. + Organization string `protobuf:"bytes,2,opt,name=organization,proto3" json:"organization,omitempty"` + // The organizational_unit in the certificate. + // Optional, default is empty. + OrganizationalUnit string `protobuf:"bytes,3,opt,name=organizational_unit,json=organizationalUnit,proto3" json:"organizational_unit,omitempty"` + // The subject_alternative_name in the certificate. + // Optional, default is empty. + SubjectAlternativeName string `protobuf:"bytes,4,opt,name=subject_alternative_name,json=subjectAlternativeName,proto3" json:"subject_alternative_name,omitempty"` +} + +func (x *CertificateFilterSpec) Reset() { + *x = CertificateFilterSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CertificateFilterSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CertificateFilterSpec) ProtoMessage() {} + +func (x *CertificateFilterSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CertificateFilterSpec.ProtoReflect.Descriptor instead. +func (*CertificateFilterSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_namespace_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *CertificateFilterSpec) GetCommonName() string { + if x != nil { + return x.CommonName + } + return "" +} + +func (x *CertificateFilterSpec) GetOrganization() string { + if x != nil { + return x.Organization + } + return "" +} + +func (x *CertificateFilterSpec) GetOrganizationalUnit() string { + if x != nil { + return x.OrganizationalUnit + } + return "" +} + +func (x *CertificateFilterSpec) GetSubjectAlternativeName() string { + if x != nil { + return x.SubjectAlternativeName + } + return "" +} + +type MtlsAuthSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The base64 encoded ca cert(s) in PEM format that the clients can use for authentication and authorization. + // This must only be one value, but the CA can have a chain. + // + // (-- api-linter: core::0140::base64=disabled --) + AcceptedClientCa string `protobuf:"bytes,1,opt,name=accepted_client_ca,json=acceptedClientCa,proto3" json:"accepted_client_ca,omitempty"` + // Certificate filters which, if specified, only allow connections from client certificates whose distinguished name properties match at least one of the filters. + // This allows limiting access to specific end-entity certificates. + // Optional, default is empty. + CertificateFilters []*CertificateFilterSpec `protobuf:"bytes,2,rep,name=certificate_filters,json=certificateFilters,proto3" json:"certificate_filters,omitempty"` + // Flag to enable mTLS auth (default: disabled). + // Note: disabling mTLS auth will cause existing mTLS connections to fail. + // temporal:versioning:min_version=2024-05-13-00 + Enabled bool `protobuf:"varint,3,opt,name=enabled,proto3" json:"enabled,omitempty"` +} + +func (x *MtlsAuthSpec) Reset() { + *x = MtlsAuthSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MtlsAuthSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MtlsAuthSpec) ProtoMessage() {} + +func (x *MtlsAuthSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MtlsAuthSpec.ProtoReflect.Descriptor instead. +func (*MtlsAuthSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_namespace_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *MtlsAuthSpec) GetAcceptedClientCa() string { + if x != nil { + return x.AcceptedClientCa + } + return "" +} + +func (x *MtlsAuthSpec) GetCertificateFilters() []*CertificateFilterSpec { + if x != nil { + return x.CertificateFilters + } + return nil +} + +func (x *MtlsAuthSpec) GetEnabled() bool { + if x != nil { + return x.Enabled + } + return false +} + +type ApiKeyAuthSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Flag to enable API key auth (default: disabled). + // Note: disabling API key auth will cause existing API key connections to fail. + Enabled bool `protobuf:"varint,1,opt,name=enabled,proto3" json:"enabled,omitempty"` +} + +func (x *ApiKeyAuthSpec) Reset() { + *x = ApiKeyAuthSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ApiKeyAuthSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ApiKeyAuthSpec) ProtoMessage() {} + +func (x *ApiKeyAuthSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ApiKeyAuthSpec.ProtoReflect.Descriptor instead. +func (*ApiKeyAuthSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_namespace_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *ApiKeyAuthSpec) GetEnabled() bool { + if x != nil { + return x.Enabled + } + return false +} + +type CodecServerSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The codec server endpoint. + Endpoint string `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + // Whether to pass the user access token with your endpoint. + PassAccessToken bool `protobuf:"varint,2,opt,name=pass_access_token,json=passAccessToken,proto3" json:"pass_access_token,omitempty"` + // Whether to include cross-origin credentials. + IncludeCrossOriginCredentials bool `protobuf:"varint,3,opt,name=include_cross_origin_credentials,json=includeCrossOriginCredentials,proto3" json:"include_cross_origin_credentials,omitempty"` +} + +func (x *CodecServerSpec) Reset() { + *x = CodecServerSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CodecServerSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CodecServerSpec) ProtoMessage() {} + +func (x *CodecServerSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CodecServerSpec.ProtoReflect.Descriptor instead. +func (*CodecServerSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_namespace_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *CodecServerSpec) GetEndpoint() string { + if x != nil { + return x.Endpoint + } + return "" +} + +func (x *CodecServerSpec) GetPassAccessToken() bool { + if x != nil { + return x.PassAccessToken + } + return false +} + +func (x *CodecServerSpec) GetIncludeCrossOriginCredentials() bool { + if x != nil { + return x.IncludeCrossOriginCredentials + } + return false +} + +type NamespaceSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The name to use for the namespace. + // This will create a namespace that's available at '..tmprl.cloud:7233'. + // The name is immutable. Once set, it cannot be changed. + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // The ids of the regions where the namespace should be available. + // The GetRegions API can be used to get the list of valid region ids. + // Specifying more than one region makes the namespace "global", which is currently a preview only feature with restricted access. + // Please reach out to Temporal support for more information on global namespaces. + // When provisioned the global namespace will be active on the first region in the list and passive on the rest. + // Number of supported regions is 2. + // The regions is immutable. Once set, it cannot be changed. + // Example: ["aws-us-west-2"]. + Regions []string `protobuf:"bytes,2,rep,name=regions,proto3" json:"regions,omitempty"` + // The number of days the workflows data will be retained for. + // Changes to the retention period may impact your storage costs. + // Any changes to the retention period will be applied to all new running workflows. + RetentionDays int32 `protobuf:"varint,3,opt,name=retention_days,json=retentionDays,proto3" json:"retention_days,omitempty"` + // The mTLS auth configuration for the namespace. + // If unspecified, mTLS will be disabled. + MtlsAuth *MtlsAuthSpec `protobuf:"bytes,4,opt,name=mtls_auth,json=mtlsAuth,proto3" json:"mtls_auth,omitempty"` + // The API key auth configuration for the namespace. + // If unspecified, API keys will be disabled. + // temporal:versioning:min_version=2024-05-13-00 + ApiKeyAuth *ApiKeyAuthSpec `protobuf:"bytes,7,opt,name=api_key_auth,json=apiKeyAuth,proto3" json:"api_key_auth,omitempty"` + // The custom search attributes to use for the namespace. + // The name of the attribute is the key and the type is the value. + // Supported attribute types: text, keyword, int, double, bool, datetime, keyword_list. + // NOTE: currently deleting a search attribute is not supported. + // Optional, default is empty. + CustomSearchAttributes map[string]string `protobuf:"bytes,5,rep,name=custom_search_attributes,json=customSearchAttributes,proto3" json:"custom_search_attributes,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // Codec server spec used by UI to decode payloads for all users interacting with this namespace. + // Optional, default is unset. + CodecServer *CodecServerSpec `protobuf:"bytes,6,opt,name=codec_server,json=codecServer,proto3" json:"codec_server,omitempty"` +} + +func (x *NamespaceSpec) Reset() { + *x = NamespaceSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NamespaceSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NamespaceSpec) ProtoMessage() {} + +func (x *NamespaceSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NamespaceSpec.ProtoReflect.Descriptor instead. +func (*NamespaceSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_namespace_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *NamespaceSpec) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *NamespaceSpec) GetRegions() []string { + if x != nil { + return x.Regions + } + return nil +} + +func (x *NamespaceSpec) GetRetentionDays() int32 { + if x != nil { + return x.RetentionDays + } + return 0 +} + +func (x *NamespaceSpec) GetMtlsAuth() *MtlsAuthSpec { + if x != nil { + return x.MtlsAuth + } + return nil +} + +func (x *NamespaceSpec) GetApiKeyAuth() *ApiKeyAuthSpec { + if x != nil { + return x.ApiKeyAuth + } + return nil +} + +func (x *NamespaceSpec) GetCustomSearchAttributes() map[string]string { + if x != nil { + return x.CustomSearchAttributes + } + return nil +} + +func (x *NamespaceSpec) GetCodecServer() *CodecServerSpec { + if x != nil { + return x.CodecServer + } + return nil +} + +type Endpoints struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The web UI address. + WebAddress string `protobuf:"bytes,1,opt,name=web_address,json=webAddress,proto3" json:"web_address,omitempty"` + // The gRPC address for mTLS client connections (may be empty if mTLS is disabled). + MtlsGrpcAddress string `protobuf:"bytes,2,opt,name=mtls_grpc_address,json=mtlsGrpcAddress,proto3" json:"mtls_grpc_address,omitempty"` + // The gRPC address for API key client connections (may be empty if API keys are disabled). + GrpcAddress string `protobuf:"bytes,3,opt,name=grpc_address,json=grpcAddress,proto3" json:"grpc_address,omitempty"` +} + +func (x *Endpoints) Reset() { + *x = Endpoints{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Endpoints) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Endpoints) ProtoMessage() {} + +func (x *Endpoints) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Endpoints.ProtoReflect.Descriptor instead. +func (*Endpoints) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_namespace_v1_message_proto_rawDescGZIP(), []int{5} +} + +func (x *Endpoints) GetWebAddress() string { + if x != nil { + return x.WebAddress + } + return "" +} + +func (x *Endpoints) GetMtlsGrpcAddress() string { + if x != nil { + return x.MtlsGrpcAddress + } + return "" +} + +func (x *Endpoints) GetGrpcAddress() string { + if x != nil { + return x.GrpcAddress + } + return "" +} + +type Limits struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The number of actions per second (APS) that is currently allowed for the namespace. + // The namespace may be throttled if its APS exceeds the limit. + ActionsPerSecondLimit int32 `protobuf:"varint,1,opt,name=actions_per_second_limit,json=actionsPerSecondLimit,proto3" json:"actions_per_second_limit,omitempty"` +} + +func (x *Limits) Reset() { + *x = Limits{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Limits) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Limits) ProtoMessage() {} + +func (x *Limits) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Limits.ProtoReflect.Descriptor instead. +func (*Limits) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_namespace_v1_message_proto_rawDescGZIP(), []int{6} +} + +func (x *Limits) GetActionsPerSecondLimit() int32 { + if x != nil { + return x.ActionsPerSecondLimit + } + return 0 +} + +type AWSPrivateLinkInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The list of principal arns that are allowed to access the namespace on the private link. + AllowedPrincipalArns []string `protobuf:"bytes,1,rep,name=allowed_principal_arns,json=allowedPrincipalArns,proto3" json:"allowed_principal_arns,omitempty"` + // The list of vpc endpoint service names that are associated with the namespace. + VpcEndpointServiceNames []string `protobuf:"bytes,2,rep,name=vpc_endpoint_service_names,json=vpcEndpointServiceNames,proto3" json:"vpc_endpoint_service_names,omitempty"` +} + +func (x *AWSPrivateLinkInfo) Reset() { + *x = AWSPrivateLinkInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AWSPrivateLinkInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AWSPrivateLinkInfo) ProtoMessage() {} + +func (x *AWSPrivateLinkInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AWSPrivateLinkInfo.ProtoReflect.Descriptor instead. +func (*AWSPrivateLinkInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_namespace_v1_message_proto_rawDescGZIP(), []int{7} +} + +func (x *AWSPrivateLinkInfo) GetAllowedPrincipalArns() []string { + if x != nil { + return x.AllowedPrincipalArns + } + return nil +} + +func (x *AWSPrivateLinkInfo) GetVpcEndpointServiceNames() []string { + if x != nil { + return x.VpcEndpointServiceNames + } + return nil +} + +type PrivateConnectivity struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the region where the private connectivity applies. + Region string `protobuf:"bytes,1,opt,name=region,proto3" json:"region,omitempty"` + // The AWS PrivateLink info. + // This will only be set for an aws region. + AwsPrivateLink *AWSPrivateLinkInfo `protobuf:"bytes,2,opt,name=aws_private_link,json=awsPrivateLink,proto3" json:"aws_private_link,omitempty"` +} + +func (x *PrivateConnectivity) Reset() { + *x = PrivateConnectivity{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PrivateConnectivity) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PrivateConnectivity) ProtoMessage() {} + +func (x *PrivateConnectivity) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PrivateConnectivity.ProtoReflect.Descriptor instead. +func (*PrivateConnectivity) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_namespace_v1_message_proto_rawDescGZIP(), []int{8} +} + +func (x *PrivateConnectivity) GetRegion() string { + if x != nil { + return x.Region + } + return "" +} + +func (x *PrivateConnectivity) GetAwsPrivateLink() *AWSPrivateLinkInfo { + if x != nil { + return x.AwsPrivateLink + } + return nil +} + +type Namespace struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace identifier. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The current version of the namespace specification. + // The next update operation will have to include this version. + ResourceVersion string `protobuf:"bytes,2,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + // The namespace specification. + Spec *NamespaceSpec `protobuf:"bytes,3,opt,name=spec,proto3" json:"spec,omitempty"` + // The current state of the namespace. + State string `protobuf:"bytes,4,opt,name=state,proto3" json:"state,omitempty"` + // The id of the async operation that is creating/updating/deleting the namespace, if any. + AsyncOperationId string `protobuf:"bytes,5,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` + // The endpoints for the namespace. + Endpoints *Endpoints `protobuf:"bytes,6,opt,name=endpoints,proto3" json:"endpoints,omitempty"` + // The currently active region for the namespace. + ActiveRegion string `protobuf:"bytes,7,opt,name=active_region,json=activeRegion,proto3" json:"active_region,omitempty"` + // The limits set on the namespace currently. + Limits *Limits `protobuf:"bytes,8,opt,name=limits,proto3" json:"limits,omitempty"` + // The private connectivities for the namespace, if any. + PrivateConnectivities []*PrivateConnectivity `protobuf:"bytes,9,rep,name=private_connectivities,json=privateConnectivities,proto3" json:"private_connectivities,omitempty"` + // The date and time when the namespace was created. + CreatedTime *timestamppb.Timestamp `protobuf:"bytes,10,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"` + // The date and time when the namespace was last modified. + // Will not be set if the namespace has never been modified. + LastModifiedTime *timestamppb.Timestamp `protobuf:"bytes,11,opt,name=last_modified_time,json=lastModifiedTime,proto3" json:"last_modified_time,omitempty"` + // The status of each region where the namespace is available. + // The id of the region is the key and the status is the value of the map. + RegionStatus map[string]*NamespaceRegionStatus `protobuf:"bytes,12,rep,name=region_status,json=regionStatus,proto3" json:"region_status,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *Namespace) Reset() { + *x = Namespace{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Namespace) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Namespace) ProtoMessage() {} + +func (x *Namespace) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Namespace.ProtoReflect.Descriptor instead. +func (*Namespace) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_namespace_v1_message_proto_rawDescGZIP(), []int{9} +} + +func (x *Namespace) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *Namespace) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *Namespace) GetSpec() *NamespaceSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *Namespace) GetState() string { + if x != nil { + return x.State + } + return "" +} + +func (x *Namespace) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +func (x *Namespace) GetEndpoints() *Endpoints { + if x != nil { + return x.Endpoints + } + return nil +} + +func (x *Namespace) GetActiveRegion() string { + if x != nil { + return x.ActiveRegion + } + return "" +} + +func (x *Namespace) GetLimits() *Limits { + if x != nil { + return x.Limits + } + return nil +} + +func (x *Namespace) GetPrivateConnectivities() []*PrivateConnectivity { + if x != nil { + return x.PrivateConnectivities + } + return nil +} + +func (x *Namespace) GetCreatedTime() *timestamppb.Timestamp { + if x != nil { + return x.CreatedTime + } + return nil +} + +func (x *Namespace) GetLastModifiedTime() *timestamppb.Timestamp { + if x != nil { + return x.LastModifiedTime + } + return nil +} + +func (x *Namespace) GetRegionStatus() map[string]*NamespaceRegionStatus { + if x != nil { + return x.RegionStatus + } + return nil +} + +type NamespaceRegionStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The current state of the namespace region. + // Possible values: adding, active, passive, removing, failed. + // For any failed state, reach out to Temporal Cloud support for remediation. + State string `protobuf:"bytes,1,opt,name=state,proto3" json:"state,omitempty"` + // The id of the async operation that is making changes to where the namespace is available, if any. + AsyncOperationId string `protobuf:"bytes,2,opt,name=async_operation_id,json=asyncOperationId,proto3" json:"async_operation_id,omitempty"` +} + +func (x *NamespaceRegionStatus) Reset() { + *x = NamespaceRegionStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NamespaceRegionStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NamespaceRegionStatus) ProtoMessage() {} + +func (x *NamespaceRegionStatus) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NamespaceRegionStatus.ProtoReflect.Descriptor instead. +func (*NamespaceRegionStatus) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_namespace_v1_message_proto_rawDescGZIP(), []int{10} +} + +func (x *NamespaceRegionStatus) GetState() string { + if x != nil { + return x.State + } + return "" +} + +func (x *NamespaceRegionStatus) GetAsyncOperationId() string { + if x != nil { + return x.AsyncOperationId + } + return "" +} + +var File_temporal_api_cloud_namespace_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_cloud_namespace_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x2d, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2f, 0x76, 0x31, 0x2f, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x1f, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xd7, 0x01, 0x0a, 0x15, 0x43, + 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x53, + 0x70, 0x65, 0x63, 0x12, 0x23, 0x0a, 0x0b, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x4e, 0x61, + 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x26, 0x0a, 0x0c, 0x6f, 0x72, 0x67, 0x61, 0x6e, 0x69, 0x7a, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x6f, 0x72, 0x67, 0x61, + 0x6e, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x33, 0x0a, 0x13, 0x6f, + 0x72, 0x67, 0x61, 0x6e, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x75, 0x6e, 0x69, + 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x6f, 0x72, 0x67, 0x61, 0x6e, 0x69, 0x7a, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x55, 0x6e, 0x69, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3c, 0x0a, + 0x18, 0x73, 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x61, 0x6c, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x74, + 0x69, 0x76, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x16, 0x73, + 0x75, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x74, 0x69, 0x76, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xcb, 0x01, 0x0a, 0x0c, 0x4d, 0x74, 0x6c, 0x73, + 0x41, 0x75, 0x74, 0x68, 0x53, 0x70, 0x65, 0x63, 0x12, 0x30, 0x0a, 0x12, 0x61, 0x63, 0x63, 0x65, 0x70, + 0x74, 0x65, 0x64, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x61, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x10, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, 0x43, 0x6c, 0x69, 0x65, 0x6e, + 0x74, 0x43, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, 0x6b, 0x0a, 0x13, 0x63, 0x65, 0x72, 0x74, 0x69, 0x66, + 0x69, 0x63, 0x61, 0x74, 0x65, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x46, 0x69, 0x6c, + 0x74, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x52, 0x12, 0x63, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, + 0x61, 0x74, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, + 0x07, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x65, + 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x2e, 0x0a, 0x0e, 0x41, 0x70, 0x69, + 0x4b, 0x65, 0x79, 0x41, 0x75, 0x74, 0x68, 0x53, 0x70, 0x65, 0x63, 0x12, 0x1c, 0x0a, 0x07, 0x65, 0x6e, + 0x61, 0x62, 0x6c, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x65, 0x6e, 0x61, 0x62, + 0x6c, 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xae, 0x01, 0x0a, 0x0f, 0x43, 0x6f, 0x64, 0x65, 0x63, + 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x12, 0x1e, 0x0a, 0x08, 0x65, 0x6e, 0x64, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2e, 0x0a, 0x11, 0x70, 0x61, 0x73, 0x73, 0x5f, + 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x0f, 0x70, 0x61, 0x73, 0x73, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x54, 0x6f, 0x6b, 0x65, + 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x20, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, + 0x63, 0x72, 0x6f, 0x73, 0x73, 0x5f, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x5f, 0x63, 0x72, 0x65, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x1d, 0x69, 0x6e, + 0x63, 0x6c, 0x75, 0x64, 0x65, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x43, + 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0xce, 0x04, + 0x0a, 0x0d, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x16, + 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x29, 0x0a, 0x0e, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x64, + 0x61, 0x79, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, + 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x79, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4e, 0x0a, 0x09, 0x6d, 0x74, + 0x6c, 0x73, 0x5f, 0x61, 0x75, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, + 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x74, 0x6c, + 0x73, 0x41, 0x75, 0x74, 0x68, 0x53, 0x70, 0x65, 0x63, 0x52, 0x08, 0x6d, 0x74, 0x6c, 0x73, 0x41, 0x75, + 0x74, 0x68, 0x42, 0x02, 0x68, 0x00, 0x12, 0x55, 0x0a, 0x0c, 0x61, 0x70, 0x69, 0x5f, 0x6b, 0x65, 0x79, + 0x5f, 0x61, 0x75, 0x74, 0x68, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x4b, 0x65, + 0x79, 0x41, 0x75, 0x74, 0x68, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0a, 0x61, 0x70, 0x69, 0x4b, 0x65, 0x79, + 0x41, 0x75, 0x74, 0x68, 0x42, 0x02, 0x68, 0x00, 0x12, 0x88, 0x01, 0x0a, 0x18, 0x63, 0x75, 0x73, 0x74, + 0x6f, 0x6d, 0x5f, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x53, 0x65, 0x61, 0x72, + 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x16, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x57, 0x0a, 0x0c, 0x63, 0x6f, + 0x64, 0x65, 0x63, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x30, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x43, 0x6f, 0x64, 0x65, 0x63, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0b, + 0x63, 0x6f, 0x64, 0x65, 0x63, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x51, + 0x0a, 0x1b, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x18, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x87, 0x01, 0x0a, + 0x09, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x65, 0x62, + 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, + 0x65, 0x62, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2e, 0x0a, 0x11, + 0x6d, 0x74, 0x6c, 0x73, 0x5f, 0x67, 0x72, 0x70, 0x63, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x6d, 0x74, 0x6c, 0x73, 0x47, 0x72, 0x70, 0x63, 0x41, + 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x67, 0x72, 0x70, + 0x63, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x67, 0x72, 0x70, 0x63, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x45, + 0x0a, 0x06, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x12, 0x3b, 0x0a, 0x18, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x5f, 0x6c, 0x69, 0x6d, + 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x15, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x50, 0x65, 0x72, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x8f, 0x01, 0x0a, 0x12, 0x41, 0x57, 0x53, 0x50, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4c, + 0x69, 0x6e, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x38, 0x0a, 0x16, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, + 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x5f, 0x61, 0x72, 0x6e, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x14, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x50, 0x72, + 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x41, 0x72, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, + 0x0a, 0x1a, 0x76, 0x70, 0x63, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x17, 0x76, 0x70, 0x63, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x94, 0x01, 0x0a, 0x13, + 0x50, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x76, 0x69, + 0x74, 0x79, 0x12, 0x1a, 0x0a, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x61, 0x0a, 0x10, + 0x61, 0x77, 0x73, 0x5f, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x6c, 0x69, 0x6e, 0x6b, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x57, 0x53, 0x50, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4c, + 0x69, 0x6e, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0e, 0x61, 0x77, 0x73, 0x50, 0x72, 0x69, 0x76, 0x61, + 0x74, 0x65, 0x4c, 0x69, 0x6e, 0x6b, 0x42, 0x02, 0x68, 0x00, 0x22, 0x96, 0x07, 0x0a, 0x09, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x46, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, + 0x70, 0x65, 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x30, 0x0a, 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4c, + 0x0a, 0x09, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x52, 0x09, 0x65, 0x6e, 0x64, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, 0x0a, 0x0d, 0x61, 0x63, 0x74, 0x69, 0x76, + 0x65, 0x5f, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x61, + 0x63, 0x74, 0x69, 0x76, 0x65, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x43, + 0x0a, 0x06, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, + 0x6d, 0x69, 0x74, 0x73, 0x52, 0x06, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x6f, 0x0a, 0x16, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, 0x65, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x34, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, + 0x64, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, + 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, + 0x52, 0x15, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x69, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x0c, 0x63, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4c, 0x0a, 0x12, 0x6c, 0x61, 0x73, + 0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0b, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x10, + 0x6c, 0x61, 0x73, 0x74, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x65, 0x0a, 0x0d, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x0c, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x42, 0x02, 0x68, 0x00, 0x1a, 0x7f, 0x0a, 0x11, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x50, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x63, 0x0a, + 0x15, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x18, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, + 0x12, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x42, 0xb1, 0x01, 0x0a, + 0x22, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, + 0x5a, 0x2f, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x2f, 0x76, 0x31, 0x3b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0xaa, 0x02, + 0x21, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x43, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x56, 0x31, + 0xea, 0x02, 0x25, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, + 0x69, 0x3a, 0x3a, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x3a, 0x3a, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_cloud_namespace_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_cloud_namespace_v1_message_proto_rawDescData = file_temporal_api_cloud_namespace_v1_message_proto_rawDesc +) + +func file_temporal_api_cloud_namespace_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_cloud_namespace_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_cloud_namespace_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_cloud_namespace_v1_message_proto_rawDescData) + }) + return file_temporal_api_cloud_namespace_v1_message_proto_rawDescData +} + +var file_temporal_api_cloud_namespace_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 13) +var file_temporal_api_cloud_namespace_v1_message_proto_goTypes = []any{ + (*CertificateFilterSpec)(nil), // 0: temporal.api.cloud.namespace.v1.CertificateFilterSpec + (*MtlsAuthSpec)(nil), // 1: temporal.api.cloud.namespace.v1.MtlsAuthSpec + (*ApiKeyAuthSpec)(nil), // 2: temporal.api.cloud.namespace.v1.ApiKeyAuthSpec + (*CodecServerSpec)(nil), // 3: temporal.api.cloud.namespace.v1.CodecServerSpec + (*NamespaceSpec)(nil), // 4: temporal.api.cloud.namespace.v1.NamespaceSpec + (*Endpoints)(nil), // 5: temporal.api.cloud.namespace.v1.Endpoints + (*Limits)(nil), // 6: temporal.api.cloud.namespace.v1.Limits + (*AWSPrivateLinkInfo)(nil), // 7: temporal.api.cloud.namespace.v1.AWSPrivateLinkInfo + (*PrivateConnectivity)(nil), // 8: temporal.api.cloud.namespace.v1.PrivateConnectivity + (*Namespace)(nil), // 9: temporal.api.cloud.namespace.v1.Namespace + (*NamespaceRegionStatus)(nil), // 10: temporal.api.cloud.namespace.v1.NamespaceRegionStatus + nil, // 11: temporal.api.cloud.namespace.v1.NamespaceSpec.CustomSearchAttributesEntry + nil, // 12: temporal.api.cloud.namespace.v1.Namespace.RegionStatusEntry + (*timestamppb.Timestamp)(nil), // 13: google.protobuf.Timestamp +} +var file_temporal_api_cloud_namespace_v1_message_proto_depIdxs = []int32{ + 0, // 0: temporal.api.cloud.namespace.v1.MtlsAuthSpec.certificate_filters:type_name -> temporal.api.cloud.namespace.v1.CertificateFilterSpec + 1, // 1: temporal.api.cloud.namespace.v1.NamespaceSpec.mtls_auth:type_name -> temporal.api.cloud.namespace.v1.MtlsAuthSpec + 2, // 2: temporal.api.cloud.namespace.v1.NamespaceSpec.api_key_auth:type_name -> temporal.api.cloud.namespace.v1.ApiKeyAuthSpec + 11, // 3: temporal.api.cloud.namespace.v1.NamespaceSpec.custom_search_attributes:type_name -> temporal.api.cloud.namespace.v1.NamespaceSpec.CustomSearchAttributesEntry + 3, // 4: temporal.api.cloud.namespace.v1.NamespaceSpec.codec_server:type_name -> temporal.api.cloud.namespace.v1.CodecServerSpec + 7, // 5: temporal.api.cloud.namespace.v1.PrivateConnectivity.aws_private_link:type_name -> temporal.api.cloud.namespace.v1.AWSPrivateLinkInfo + 4, // 6: temporal.api.cloud.namespace.v1.Namespace.spec:type_name -> temporal.api.cloud.namespace.v1.NamespaceSpec + 5, // 7: temporal.api.cloud.namespace.v1.Namespace.endpoints:type_name -> temporal.api.cloud.namespace.v1.Endpoints + 6, // 8: temporal.api.cloud.namespace.v1.Namespace.limits:type_name -> temporal.api.cloud.namespace.v1.Limits + 8, // 9: temporal.api.cloud.namespace.v1.Namespace.private_connectivities:type_name -> temporal.api.cloud.namespace.v1.PrivateConnectivity + 13, // 10: temporal.api.cloud.namespace.v1.Namespace.created_time:type_name -> google.protobuf.Timestamp + 13, // 11: temporal.api.cloud.namespace.v1.Namespace.last_modified_time:type_name -> google.protobuf.Timestamp + 12, // 12: temporal.api.cloud.namespace.v1.Namespace.region_status:type_name -> temporal.api.cloud.namespace.v1.Namespace.RegionStatusEntry + 10, // 13: temporal.api.cloud.namespace.v1.Namespace.RegionStatusEntry.value:type_name -> temporal.api.cloud.namespace.v1.NamespaceRegionStatus + 14, // [14:14] is the sub-list for method output_type + 14, // [14:14] is the sub-list for method input_type + 14, // [14:14] is the sub-list for extension type_name + 14, // [14:14] is the sub-list for extension extendee + 0, // [0:14] is the sub-list for field type_name +} + +func init() { file_temporal_api_cloud_namespace_v1_message_proto_init() } +func file_temporal_api_cloud_namespace_v1_message_proto_init() { + if File_temporal_api_cloud_namespace_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*CertificateFilterSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*MtlsAuthSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*ApiKeyAuthSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*CodecServerSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*NamespaceSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*Endpoints); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*Limits); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*AWSPrivateLinkInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*PrivateConnectivity); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*Namespace); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_cloud_namespace_v1_message_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*NamespaceRegionStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_cloud_namespace_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 13, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_cloud_namespace_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_cloud_namespace_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_cloud_namespace_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_cloud_namespace_v1_message_proto = out.File + file_temporal_api_cloud_namespace_v1_message_proto_rawDesc = nil + file_temporal_api_cloud_namespace_v1_message_proto_goTypes = nil + file_temporal_api_cloud_namespace_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/cloud/operation/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/cloud/operation/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..0a6faa9e492 --- /dev/null +++ b/vendor/go.temporal.io/api/cloud/operation/v1/message.go-helpers.pb.go @@ -0,0 +1,65 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package operation + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type AsyncOperation to the protobuf v3 wire format +func (val *AsyncOperation) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type AsyncOperation from the protobuf v3 wire format +func (val *AsyncOperation) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *AsyncOperation) Size() int { + return proto.Size(val) +} + +// Equal returns whether two AsyncOperation values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *AsyncOperation) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *AsyncOperation + switch t := that.(type) { + case *AsyncOperation: + that1 = t + case AsyncOperation: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/cloud/operation/v1/message.pb.go b/vendor/go.temporal.io/api/cloud/operation/v1/message.pb.go new file mode 100644 index 00000000000..16e59e54d61 --- /dev/null +++ b/vendor/go.temporal.io/api/cloud/operation/v1/message.pb.go @@ -0,0 +1,283 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/cloud/operation/v1/message.proto + +package operation + +import ( + reflect "reflect" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + anypb "google.golang.org/protobuf/types/known/anypb" + durationpb "google.golang.org/protobuf/types/known/durationpb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type AsyncOperation struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The operation id + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + // The current state of this operation + // Possible values are: pending, in_progress, failed, cancelled, fulfilled + State string `protobuf:"bytes,2,opt,name=state,proto3" json:"state,omitempty"` + // The recommended duration to check back for an update in the operation's state + CheckDuration *durationpb.Duration `protobuf:"bytes,3,opt,name=check_duration,json=checkDuration,proto3" json:"check_duration,omitempty"` + // The type of operation being performed + OperationType string `protobuf:"bytes,4,opt,name=operation_type,json=operationType,proto3" json:"operation_type,omitempty"` + // The input to the operation being performed + // + // (-- api-linter: core::0146::any=disabled --) + OperationInput *anypb.Any `protobuf:"bytes,5,opt,name=operation_input,json=operationInput,proto3" json:"operation_input,omitempty"` + // If the operation failed, the reason for the failure + FailureReason string `protobuf:"bytes,6,opt,name=failure_reason,json=failureReason,proto3" json:"failure_reason,omitempty"` + // The date and time when the operation initiated + StartedTime *timestamppb.Timestamp `protobuf:"bytes,7,opt,name=started_time,json=startedTime,proto3" json:"started_time,omitempty"` + // The date and time when the operation completed + FinishedTime *timestamppb.Timestamp `protobuf:"bytes,8,opt,name=finished_time,json=finishedTime,proto3" json:"finished_time,omitempty"` +} + +func (x *AsyncOperation) Reset() { + *x = AsyncOperation{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_operation_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AsyncOperation) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AsyncOperation) ProtoMessage() {} + +func (x *AsyncOperation) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_operation_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AsyncOperation.ProtoReflect.Descriptor instead. +func (*AsyncOperation) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_operation_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *AsyncOperation) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *AsyncOperation) GetState() string { + if x != nil { + return x.State + } + return "" +} + +func (x *AsyncOperation) GetCheckDuration() *durationpb.Duration { + if x != nil { + return x.CheckDuration + } + return nil +} + +func (x *AsyncOperation) GetOperationType() string { + if x != nil { + return x.OperationType + } + return "" +} + +func (x *AsyncOperation) GetOperationInput() *anypb.Any { + if x != nil { + return x.OperationInput + } + return nil +} + +func (x *AsyncOperation) GetFailureReason() string { + if x != nil { + return x.FailureReason + } + return "" +} + +func (x *AsyncOperation) GetStartedTime() *timestamppb.Timestamp { + if x != nil { + return x.StartedTime + } + return nil +} + +func (x *AsyncOperation) GetFinishedTime() *timestamppb.Timestamp { + if x != nil { + return x.FinishedTime + } + return nil +} + +var File_temporal_api_cloud_operation_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_cloud_operation_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x2d, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x2f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x1f, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, + 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x1a, 0x1e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x75, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x19, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x61, 0x6e, 0x79, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xa5, 0x03, 0x0a, 0x0e, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x44, 0x0a, 0x0e, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x5f, 0x64, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, 0x0a, 0x0e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x41, 0x0a, 0x0f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x70, + 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x52, 0x0e, 0x6f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x29, 0x0a, 0x0e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, + 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, + 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, + 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x43, 0x0a, 0x0d, 0x66, 0x69, 0x6e, + 0x69, 0x73, 0x68, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0c, 0x66, 0x69, 0x6e, 0x69, + 0x73, 0x68, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x42, 0xb1, 0x01, 0x0a, 0x22, + 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, + 0x5a, 0x2f, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x3b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0xaa, + 0x02, 0x21, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, + 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, + 0x56, 0x31, 0xea, 0x02, 0x25, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, + 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x3a, 0x3a, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_cloud_operation_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_cloud_operation_v1_message_proto_rawDescData = file_temporal_api_cloud_operation_v1_message_proto_rawDesc +) + +func file_temporal_api_cloud_operation_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_cloud_operation_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_cloud_operation_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_cloud_operation_v1_message_proto_rawDescData) + }) + return file_temporal_api_cloud_operation_v1_message_proto_rawDescData +} + +var file_temporal_api_cloud_operation_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 1) +var file_temporal_api_cloud_operation_v1_message_proto_goTypes = []any{ + (*AsyncOperation)(nil), // 0: temporal.api.cloud.operation.v1.AsyncOperation + (*durationpb.Duration)(nil), // 1: google.protobuf.Duration + (*anypb.Any)(nil), // 2: google.protobuf.Any + (*timestamppb.Timestamp)(nil), // 3: google.protobuf.Timestamp +} +var file_temporal_api_cloud_operation_v1_message_proto_depIdxs = []int32{ + 1, // 0: temporal.api.cloud.operation.v1.AsyncOperation.check_duration:type_name -> google.protobuf.Duration + 2, // 1: temporal.api.cloud.operation.v1.AsyncOperation.operation_input:type_name -> google.protobuf.Any + 3, // 2: temporal.api.cloud.operation.v1.AsyncOperation.started_time:type_name -> google.protobuf.Timestamp + 3, // 3: temporal.api.cloud.operation.v1.AsyncOperation.finished_time:type_name -> google.protobuf.Timestamp + 4, // [4:4] is the sub-list for method output_type + 4, // [4:4] is the sub-list for method input_type + 4, // [4:4] is the sub-list for extension type_name + 4, // [4:4] is the sub-list for extension extendee + 0, // [0:4] is the sub-list for field type_name +} + +func init() { file_temporal_api_cloud_operation_v1_message_proto_init() } +func file_temporal_api_cloud_operation_v1_message_proto_init() { + if File_temporal_api_cloud_operation_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_cloud_operation_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*AsyncOperation); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_cloud_operation_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 1, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_cloud_operation_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_cloud_operation_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_cloud_operation_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_cloud_operation_v1_message_proto = out.File + file_temporal_api_cloud_operation_v1_message_proto_rawDesc = nil + file_temporal_api_cloud_operation_v1_message_proto_goTypes = nil + file_temporal_api_cloud_operation_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/cloud/region/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/cloud/region/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..20924bca14d --- /dev/null +++ b/vendor/go.temporal.io/api/cloud/region/v1/message.go-helpers.pb.go @@ -0,0 +1,65 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package region + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type Region to the protobuf v3 wire format +func (val *Region) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Region from the protobuf v3 wire format +func (val *Region) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Region) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Region values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Region) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Region + switch t := that.(type) { + case *Region: + that1 = t + case Region: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/cloud/region/v1/message.pb.go b/vendor/go.temporal.io/api/cloud/region/v1/message.pb.go new file mode 100644 index 00000000000..504ae25b1c4 --- /dev/null +++ b/vendor/go.temporal.io/api/cloud/region/v1/message.pb.go @@ -0,0 +1,213 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/cloud/region/v1/message.proto + +package region + +import ( + reflect "reflect" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type Region struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the temporal cloud region. + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + // The name of the cloud provider that's hosting the region. + // Currently only "aws" is supported. + CloudProvider string `protobuf:"bytes,2,opt,name=cloud_provider,json=cloudProvider,proto3" json:"cloud_provider,omitempty"` + // The region identifier as defined by the cloud provider. + CloudProviderRegion string `protobuf:"bytes,3,opt,name=cloud_provider_region,json=cloudProviderRegion,proto3" json:"cloud_provider_region,omitempty"` + // The human readable location of the region. + Location string `protobuf:"bytes,4,opt,name=location,proto3" json:"location,omitempty"` +} + +func (x *Region) Reset() { + *x = Region{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_cloud_region_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Region) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Region) ProtoMessage() {} + +func (x *Region) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_cloud_region_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Region.ProtoReflect.Descriptor instead. +func (*Region) Descriptor() ([]byte, []int) { + return file_temporal_api_cloud_region_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *Region) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *Region) GetCloudProvider() string { + if x != nil { + return x.CloudProvider + } + return "" +} + +func (x *Region) GetCloudProviderRegion() string { + if x != nil { + return x.CloudProviderRegion + } + return "" +} + +func (x *Region) GetLocation() string { + if x != nil { + return x.Location + } + return "" +} + +var File_temporal_api_cloud_region_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_cloud_region_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x2a, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, + 0x6f, 0x75, 0x64, 0x2f, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x1c, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x72, 0x65, + 0x67, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x22, 0x9f, 0x01, 0x0a, 0x06, 0x52, 0x65, 0x67, 0x69, 0x6f, + 0x6e, 0x12, 0x12, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x5f, 0x70, 0x72, + 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x36, + 0x0a, 0x15, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x5f, + 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x63, 0x6c, 0x6f, + 0x75, 0x64, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x42, 0xa2, 0x01, 0x0a, 0x1f, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x72, 0x65, 0x67, 0x69, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, + 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x29, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, 0x6f, 0x75, 0x64, 0x2f, 0x72, 0x65, + 0x67, 0x69, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x3b, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0xaa, 0x02, 0x1e, + 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x43, + 0x6c, 0x6f, 0x75, 0x64, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x22, + 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, + 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x3a, 0x3a, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x3a, 0x3a, 0x56, + 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_cloud_region_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_cloud_region_v1_message_proto_rawDescData = file_temporal_api_cloud_region_v1_message_proto_rawDesc +) + +func file_temporal_api_cloud_region_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_cloud_region_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_cloud_region_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_cloud_region_v1_message_proto_rawDescData) + }) + return file_temporal_api_cloud_region_v1_message_proto_rawDescData +} + +var file_temporal_api_cloud_region_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 1) +var file_temporal_api_cloud_region_v1_message_proto_goTypes = []any{ + (*Region)(nil), // 0: temporal.api.cloud.region.v1.Region +} +var file_temporal_api_cloud_region_v1_message_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_cloud_region_v1_message_proto_init() } +func file_temporal_api_cloud_region_v1_message_proto_init() { + if File_temporal_api_cloud_region_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_cloud_region_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*Region); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_cloud_region_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 1, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_cloud_region_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_cloud_region_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_cloud_region_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_cloud_region_v1_message_proto = out.File + file_temporal_api_cloud_region_v1_message_proto_rawDesc = nil + file_temporal_api_cloud_region_v1_message_proto_goTypes = nil + file_temporal_api_cloud_region_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/command/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/command/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..0e62294c6ae --- /dev/null +++ b/vendor/go.temporal.io/api/command/v1/message.go-helpers.pb.go @@ -0,0 +1,694 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package command + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type ScheduleActivityTaskCommandAttributes to the protobuf v3 wire format +func (val *ScheduleActivityTaskCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ScheduleActivityTaskCommandAttributes from the protobuf v3 wire format +func (val *ScheduleActivityTaskCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ScheduleActivityTaskCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ScheduleActivityTaskCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ScheduleActivityTaskCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ScheduleActivityTaskCommandAttributes + switch t := that.(type) { + case *ScheduleActivityTaskCommandAttributes: + that1 = t + case ScheduleActivityTaskCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RequestCancelActivityTaskCommandAttributes to the protobuf v3 wire format +func (val *RequestCancelActivityTaskCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RequestCancelActivityTaskCommandAttributes from the protobuf v3 wire format +func (val *RequestCancelActivityTaskCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RequestCancelActivityTaskCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RequestCancelActivityTaskCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RequestCancelActivityTaskCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RequestCancelActivityTaskCommandAttributes + switch t := that.(type) { + case *RequestCancelActivityTaskCommandAttributes: + that1 = t + case RequestCancelActivityTaskCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StartTimerCommandAttributes to the protobuf v3 wire format +func (val *StartTimerCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StartTimerCommandAttributes from the protobuf v3 wire format +func (val *StartTimerCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StartTimerCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StartTimerCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StartTimerCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StartTimerCommandAttributes + switch t := that.(type) { + case *StartTimerCommandAttributes: + that1 = t + case StartTimerCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CompleteWorkflowExecutionCommandAttributes to the protobuf v3 wire format +func (val *CompleteWorkflowExecutionCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CompleteWorkflowExecutionCommandAttributes from the protobuf v3 wire format +func (val *CompleteWorkflowExecutionCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CompleteWorkflowExecutionCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CompleteWorkflowExecutionCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CompleteWorkflowExecutionCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CompleteWorkflowExecutionCommandAttributes + switch t := that.(type) { + case *CompleteWorkflowExecutionCommandAttributes: + that1 = t + case CompleteWorkflowExecutionCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type FailWorkflowExecutionCommandAttributes to the protobuf v3 wire format +func (val *FailWorkflowExecutionCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type FailWorkflowExecutionCommandAttributes from the protobuf v3 wire format +func (val *FailWorkflowExecutionCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *FailWorkflowExecutionCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two FailWorkflowExecutionCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *FailWorkflowExecutionCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *FailWorkflowExecutionCommandAttributes + switch t := that.(type) { + case *FailWorkflowExecutionCommandAttributes: + that1 = t + case FailWorkflowExecutionCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CancelTimerCommandAttributes to the protobuf v3 wire format +func (val *CancelTimerCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CancelTimerCommandAttributes from the protobuf v3 wire format +func (val *CancelTimerCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CancelTimerCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CancelTimerCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CancelTimerCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CancelTimerCommandAttributes + switch t := that.(type) { + case *CancelTimerCommandAttributes: + that1 = t + case CancelTimerCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CancelWorkflowExecutionCommandAttributes to the protobuf v3 wire format +func (val *CancelWorkflowExecutionCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CancelWorkflowExecutionCommandAttributes from the protobuf v3 wire format +func (val *CancelWorkflowExecutionCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CancelWorkflowExecutionCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CancelWorkflowExecutionCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CancelWorkflowExecutionCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CancelWorkflowExecutionCommandAttributes + switch t := that.(type) { + case *CancelWorkflowExecutionCommandAttributes: + that1 = t + case CancelWorkflowExecutionCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RequestCancelExternalWorkflowExecutionCommandAttributes to the protobuf v3 wire format +func (val *RequestCancelExternalWorkflowExecutionCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RequestCancelExternalWorkflowExecutionCommandAttributes from the protobuf v3 wire format +func (val *RequestCancelExternalWorkflowExecutionCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RequestCancelExternalWorkflowExecutionCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RequestCancelExternalWorkflowExecutionCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RequestCancelExternalWorkflowExecutionCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RequestCancelExternalWorkflowExecutionCommandAttributes + switch t := that.(type) { + case *RequestCancelExternalWorkflowExecutionCommandAttributes: + that1 = t + case RequestCancelExternalWorkflowExecutionCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SignalExternalWorkflowExecutionCommandAttributes to the protobuf v3 wire format +func (val *SignalExternalWorkflowExecutionCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SignalExternalWorkflowExecutionCommandAttributes from the protobuf v3 wire format +func (val *SignalExternalWorkflowExecutionCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SignalExternalWorkflowExecutionCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SignalExternalWorkflowExecutionCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SignalExternalWorkflowExecutionCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SignalExternalWorkflowExecutionCommandAttributes + switch t := that.(type) { + case *SignalExternalWorkflowExecutionCommandAttributes: + that1 = t + case SignalExternalWorkflowExecutionCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpsertWorkflowSearchAttributesCommandAttributes to the protobuf v3 wire format +func (val *UpsertWorkflowSearchAttributesCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpsertWorkflowSearchAttributesCommandAttributes from the protobuf v3 wire format +func (val *UpsertWorkflowSearchAttributesCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpsertWorkflowSearchAttributesCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpsertWorkflowSearchAttributesCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpsertWorkflowSearchAttributesCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpsertWorkflowSearchAttributesCommandAttributes + switch t := that.(type) { + case *UpsertWorkflowSearchAttributesCommandAttributes: + that1 = t + case UpsertWorkflowSearchAttributesCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ModifyWorkflowPropertiesCommandAttributes to the protobuf v3 wire format +func (val *ModifyWorkflowPropertiesCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ModifyWorkflowPropertiesCommandAttributes from the protobuf v3 wire format +func (val *ModifyWorkflowPropertiesCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ModifyWorkflowPropertiesCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ModifyWorkflowPropertiesCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ModifyWorkflowPropertiesCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ModifyWorkflowPropertiesCommandAttributes + switch t := that.(type) { + case *ModifyWorkflowPropertiesCommandAttributes: + that1 = t + case ModifyWorkflowPropertiesCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RecordMarkerCommandAttributes to the protobuf v3 wire format +func (val *RecordMarkerCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RecordMarkerCommandAttributes from the protobuf v3 wire format +func (val *RecordMarkerCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RecordMarkerCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RecordMarkerCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RecordMarkerCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RecordMarkerCommandAttributes + switch t := that.(type) { + case *RecordMarkerCommandAttributes: + that1 = t + case RecordMarkerCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ContinueAsNewWorkflowExecutionCommandAttributes to the protobuf v3 wire format +func (val *ContinueAsNewWorkflowExecutionCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ContinueAsNewWorkflowExecutionCommandAttributes from the protobuf v3 wire format +func (val *ContinueAsNewWorkflowExecutionCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ContinueAsNewWorkflowExecutionCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ContinueAsNewWorkflowExecutionCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ContinueAsNewWorkflowExecutionCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ContinueAsNewWorkflowExecutionCommandAttributes + switch t := that.(type) { + case *ContinueAsNewWorkflowExecutionCommandAttributes: + that1 = t + case ContinueAsNewWorkflowExecutionCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StartChildWorkflowExecutionCommandAttributes to the protobuf v3 wire format +func (val *StartChildWorkflowExecutionCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StartChildWorkflowExecutionCommandAttributes from the protobuf v3 wire format +func (val *StartChildWorkflowExecutionCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StartChildWorkflowExecutionCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StartChildWorkflowExecutionCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StartChildWorkflowExecutionCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StartChildWorkflowExecutionCommandAttributes + switch t := that.(type) { + case *StartChildWorkflowExecutionCommandAttributes: + that1 = t + case StartChildWorkflowExecutionCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ProtocolMessageCommandAttributes to the protobuf v3 wire format +func (val *ProtocolMessageCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ProtocolMessageCommandAttributes from the protobuf v3 wire format +func (val *ProtocolMessageCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ProtocolMessageCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ProtocolMessageCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ProtocolMessageCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ProtocolMessageCommandAttributes + switch t := that.(type) { + case *ProtocolMessageCommandAttributes: + that1 = t + case ProtocolMessageCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ScheduleNexusOperationCommandAttributes to the protobuf v3 wire format +func (val *ScheduleNexusOperationCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ScheduleNexusOperationCommandAttributes from the protobuf v3 wire format +func (val *ScheduleNexusOperationCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ScheduleNexusOperationCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ScheduleNexusOperationCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ScheduleNexusOperationCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ScheduleNexusOperationCommandAttributes + switch t := that.(type) { + case *ScheduleNexusOperationCommandAttributes: + that1 = t + case ScheduleNexusOperationCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RequestCancelNexusOperationCommandAttributes to the protobuf v3 wire format +func (val *RequestCancelNexusOperationCommandAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RequestCancelNexusOperationCommandAttributes from the protobuf v3 wire format +func (val *RequestCancelNexusOperationCommandAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RequestCancelNexusOperationCommandAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RequestCancelNexusOperationCommandAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RequestCancelNexusOperationCommandAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RequestCancelNexusOperationCommandAttributes + switch t := that.(type) { + case *RequestCancelNexusOperationCommandAttributes: + that1 = t + case RequestCancelNexusOperationCommandAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Command to the protobuf v3 wire format +func (val *Command) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Command from the protobuf v3 wire format +func (val *Command) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Command) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Command values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Command) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Command + switch t := that.(type) { + case *Command: + that1 = t + case Command: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/command/v1/message.pb.go b/vendor/go.temporal.io/api/command/v1/message.pb.go new file mode 100644 index 00000000000..cfc0c31c8b1 --- /dev/null +++ b/vendor/go.temporal.io/api/command/v1/message.pb.go @@ -0,0 +1,2709 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/command/v1/message.proto + +package command + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/common/v1" + v13 "go.temporal.io/api/enums/v1" + v12 "go.temporal.io/api/failure/v1" + v14 "go.temporal.io/api/sdk/v1" + v11 "go.temporal.io/api/taskqueue/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + durationpb "google.golang.org/protobuf/types/known/durationpb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type ScheduleActivityTaskCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ActivityId string `protobuf:"bytes,1,opt,name=activity_id,json=activityId,proto3" json:"activity_id,omitempty"` + ActivityType *v1.ActivityType `protobuf:"bytes,2,opt,name=activity_type,json=activityType,proto3" json:"activity_type,omitempty"` + TaskQueue *v11.TaskQueue `protobuf:"bytes,4,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + Header *v1.Header `protobuf:"bytes,5,opt,name=header,proto3" json:"header,omitempty"` + Input *v1.Payloads `protobuf:"bytes,6,opt,name=input,proto3" json:"input,omitempty"` + // Indicates how long the caller is willing to wait for activity completion. The "schedule" time + // is when the activity is initially scheduled, not when the most recent retry is scheduled. + // Limits how long retries will be attempted. Either this or `start_to_close_timeout` must be + // specified. When not specified, defaults to the workflow execution timeout. + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + ScheduleToCloseTimeout *durationpb.Duration `protobuf:"bytes,7,opt,name=schedule_to_close_timeout,json=scheduleToCloseTimeout,proto3" json:"schedule_to_close_timeout,omitempty"` + // Limits the time an activity task can stay in a task queue before a worker picks it up. The + // "schedule" time is when the most recent retry is scheduled. This timeout should usually not + // be set: it's useful in specific scenarios like worker-specific task queues. This timeout is + // always non retryable, as all a retry would achieve is to put it back into the same queue. + // Defaults to `schedule_to_close_timeout` or workflow execution timeout if that is not + // specified. More info: + // https://docs.temporal.io/docs/content/what-is-a-schedule-to-start-timeout/ + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + ScheduleToStartTimeout *durationpb.Duration `protobuf:"bytes,8,opt,name=schedule_to_start_timeout,json=scheduleToStartTimeout,proto3" json:"schedule_to_start_timeout,omitempty"` + // Maximum time an activity is allowed to execute after being picked up by a worker. This + // timeout is always retryable. Either this or `schedule_to_close_timeout` must be specified. + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + StartToCloseTimeout *durationpb.Duration `protobuf:"bytes,9,opt,name=start_to_close_timeout,json=startToCloseTimeout,proto3" json:"start_to_close_timeout,omitempty"` + // Maximum permitted time between successful worker heartbeats. + HeartbeatTimeout *durationpb.Duration `protobuf:"bytes,10,opt,name=heartbeat_timeout,json=heartbeatTimeout,proto3" json:"heartbeat_timeout,omitempty"` + // Activities are provided by a default retry policy which is controlled through the service's + // dynamic configuration. Retries will be attempted until `schedule_to_close_timeout` has + // elapsed. To disable retries set retry_policy.maximum_attempts to 1. + RetryPolicy *v1.RetryPolicy `protobuf:"bytes,11,opt,name=retry_policy,json=retryPolicy,proto3" json:"retry_policy,omitempty"` + // Request to start the activity directly bypassing matching service and worker polling + // The slot for executing the activity should be reserved when setting this field to true. + RequestEagerExecution bool `protobuf:"varint,12,opt,name=request_eager_execution,json=requestEagerExecution,proto3" json:"request_eager_execution,omitempty"` + // If this is set, the activity would be assigned to the Build ID of the workflow. Otherwise, + // Assignment rules of the activity's Task Queue will be used to determine the Build ID. + UseWorkflowBuildId bool `protobuf:"varint,13,opt,name=use_workflow_build_id,json=useWorkflowBuildId,proto3" json:"use_workflow_build_id,omitempty"` +} + +func (x *ScheduleActivityTaskCommandAttributes) Reset() { + *x = ScheduleActivityTaskCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ScheduleActivityTaskCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ScheduleActivityTaskCommandAttributes) ProtoMessage() {} + +func (x *ScheduleActivityTaskCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ScheduleActivityTaskCommandAttributes.ProtoReflect.Descriptor instead. +func (*ScheduleActivityTaskCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *ScheduleActivityTaskCommandAttributes) GetActivityId() string { + if x != nil { + return x.ActivityId + } + return "" +} + +func (x *ScheduleActivityTaskCommandAttributes) GetActivityType() *v1.ActivityType { + if x != nil { + return x.ActivityType + } + return nil +} + +func (x *ScheduleActivityTaskCommandAttributes) GetTaskQueue() *v11.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *ScheduleActivityTaskCommandAttributes) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *ScheduleActivityTaskCommandAttributes) GetInput() *v1.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *ScheduleActivityTaskCommandAttributes) GetScheduleToCloseTimeout() *durationpb.Duration { + if x != nil { + return x.ScheduleToCloseTimeout + } + return nil +} + +func (x *ScheduleActivityTaskCommandAttributes) GetScheduleToStartTimeout() *durationpb.Duration { + if x != nil { + return x.ScheduleToStartTimeout + } + return nil +} + +func (x *ScheduleActivityTaskCommandAttributes) GetStartToCloseTimeout() *durationpb.Duration { + if x != nil { + return x.StartToCloseTimeout + } + return nil +} + +func (x *ScheduleActivityTaskCommandAttributes) GetHeartbeatTimeout() *durationpb.Duration { + if x != nil { + return x.HeartbeatTimeout + } + return nil +} + +func (x *ScheduleActivityTaskCommandAttributes) GetRetryPolicy() *v1.RetryPolicy { + if x != nil { + return x.RetryPolicy + } + return nil +} + +func (x *ScheduleActivityTaskCommandAttributes) GetRequestEagerExecution() bool { + if x != nil { + return x.RequestEagerExecution + } + return false +} + +func (x *ScheduleActivityTaskCommandAttributes) GetUseWorkflowBuildId() bool { + if x != nil { + return x.UseWorkflowBuildId + } + return false +} + +type RequestCancelActivityTaskCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The `ACTIVITY_TASK_SCHEDULED` event id for the activity being cancelled. + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` +} + +func (x *RequestCancelActivityTaskCommandAttributes) Reset() { + *x = RequestCancelActivityTaskCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RequestCancelActivityTaskCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RequestCancelActivityTaskCommandAttributes) ProtoMessage() {} + +func (x *RequestCancelActivityTaskCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RequestCancelActivityTaskCommandAttributes.ProtoReflect.Descriptor instead. +func (*RequestCancelActivityTaskCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *RequestCancelActivityTaskCommandAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +type StartTimerCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // An id for the timer, currently live timers must have different ids. Typically autogenerated + // by the SDK. + TimerId string `protobuf:"bytes,1,opt,name=timer_id,json=timerId,proto3" json:"timer_id,omitempty"` + // How long until the timer fires, producing a `TIMER_FIRED` event. + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + StartToFireTimeout *durationpb.Duration `protobuf:"bytes,2,opt,name=start_to_fire_timeout,json=startToFireTimeout,proto3" json:"start_to_fire_timeout,omitempty"` +} + +func (x *StartTimerCommandAttributes) Reset() { + *x = StartTimerCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartTimerCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartTimerCommandAttributes) ProtoMessage() {} + +func (x *StartTimerCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartTimerCommandAttributes.ProtoReflect.Descriptor instead. +func (*StartTimerCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *StartTimerCommandAttributes) GetTimerId() string { + if x != nil { + return x.TimerId + } + return "" +} + +func (x *StartTimerCommandAttributes) GetStartToFireTimeout() *durationpb.Duration { + if x != nil { + return x.StartToFireTimeout + } + return nil +} + +type CompleteWorkflowExecutionCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Result *v1.Payloads `protobuf:"bytes,1,opt,name=result,proto3" json:"result,omitempty"` +} + +func (x *CompleteWorkflowExecutionCommandAttributes) Reset() { + *x = CompleteWorkflowExecutionCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CompleteWorkflowExecutionCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CompleteWorkflowExecutionCommandAttributes) ProtoMessage() {} + +func (x *CompleteWorkflowExecutionCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CompleteWorkflowExecutionCommandAttributes.ProtoReflect.Descriptor instead. +func (*CompleteWorkflowExecutionCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *CompleteWorkflowExecutionCommandAttributes) GetResult() *v1.Payloads { + if x != nil { + return x.Result + } + return nil +} + +type FailWorkflowExecutionCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Failure *v12.Failure `protobuf:"bytes,1,opt,name=failure,proto3" json:"failure,omitempty"` +} + +func (x *FailWorkflowExecutionCommandAttributes) Reset() { + *x = FailWorkflowExecutionCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FailWorkflowExecutionCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FailWorkflowExecutionCommandAttributes) ProtoMessage() {} + +func (x *FailWorkflowExecutionCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FailWorkflowExecutionCommandAttributes.ProtoReflect.Descriptor instead. +func (*FailWorkflowExecutionCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *FailWorkflowExecutionCommandAttributes) GetFailure() *v12.Failure { + if x != nil { + return x.Failure + } + return nil +} + +type CancelTimerCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The same timer id from the start timer command + TimerId string `protobuf:"bytes,1,opt,name=timer_id,json=timerId,proto3" json:"timer_id,omitempty"` +} + +func (x *CancelTimerCommandAttributes) Reset() { + *x = CancelTimerCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CancelTimerCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CancelTimerCommandAttributes) ProtoMessage() {} + +func (x *CancelTimerCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CancelTimerCommandAttributes.ProtoReflect.Descriptor instead. +func (*CancelTimerCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{5} +} + +func (x *CancelTimerCommandAttributes) GetTimerId() string { + if x != nil { + return x.TimerId + } + return "" +} + +type CancelWorkflowExecutionCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Details *v1.Payloads `protobuf:"bytes,1,opt,name=details,proto3" json:"details,omitempty"` +} + +func (x *CancelWorkflowExecutionCommandAttributes) Reset() { + *x = CancelWorkflowExecutionCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CancelWorkflowExecutionCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CancelWorkflowExecutionCommandAttributes) ProtoMessage() {} + +func (x *CancelWorkflowExecutionCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CancelWorkflowExecutionCommandAttributes.ProtoReflect.Descriptor instead. +func (*CancelWorkflowExecutionCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{6} +} + +func (x *CancelWorkflowExecutionCommandAttributes) GetDetails() *v1.Payloads { + if x != nil { + return x.Details + } + return nil +} + +type RequestCancelExternalWorkflowExecutionCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + WorkflowId string `protobuf:"bytes,2,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + RunId string `protobuf:"bytes,3,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` + // Deprecated. + Control string `protobuf:"bytes,4,opt,name=control,proto3" json:"control,omitempty"` + // Set this to true if the workflow being cancelled is a child of the workflow originating this + // command. The request will be rejected if it is set to true and the target workflow is *not* + // a child of the requesting workflow. + ChildWorkflowOnly bool `protobuf:"varint,5,opt,name=child_workflow_only,json=childWorkflowOnly,proto3" json:"child_workflow_only,omitempty"` + // Reason for requesting the cancellation + Reason string `protobuf:"bytes,6,opt,name=reason,proto3" json:"reason,omitempty"` +} + +func (x *RequestCancelExternalWorkflowExecutionCommandAttributes) Reset() { + *x = RequestCancelExternalWorkflowExecutionCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RequestCancelExternalWorkflowExecutionCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RequestCancelExternalWorkflowExecutionCommandAttributes) ProtoMessage() {} + +func (x *RequestCancelExternalWorkflowExecutionCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RequestCancelExternalWorkflowExecutionCommandAttributes.ProtoReflect.Descriptor instead. +func (*RequestCancelExternalWorkflowExecutionCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{7} +} + +func (x *RequestCancelExternalWorkflowExecutionCommandAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RequestCancelExternalWorkflowExecutionCommandAttributes) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *RequestCancelExternalWorkflowExecutionCommandAttributes) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +func (x *RequestCancelExternalWorkflowExecutionCommandAttributes) GetControl() string { + if x != nil { + return x.Control + } + return "" +} + +func (x *RequestCancelExternalWorkflowExecutionCommandAttributes) GetChildWorkflowOnly() bool { + if x != nil { + return x.ChildWorkflowOnly + } + return false +} + +func (x *RequestCancelExternalWorkflowExecutionCommandAttributes) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +type SignalExternalWorkflowExecutionCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + Execution *v1.WorkflowExecution `protobuf:"bytes,2,opt,name=execution,proto3" json:"execution,omitempty"` + // The workflow author-defined name of the signal to send to the workflow. + SignalName string `protobuf:"bytes,3,opt,name=signal_name,json=signalName,proto3" json:"signal_name,omitempty"` + // Serialized value(s) to provide with the signal. + Input *v1.Payloads `protobuf:"bytes,4,opt,name=input,proto3" json:"input,omitempty"` + // Deprecated + Control string `protobuf:"bytes,5,opt,name=control,proto3" json:"control,omitempty"` + // Set this to true if the workflow being cancelled is a child of the workflow originating this + // command. The request will be rejected if it is set to true and the target workflow is *not* + // a child of the requesting workflow. + ChildWorkflowOnly bool `protobuf:"varint,6,opt,name=child_workflow_only,json=childWorkflowOnly,proto3" json:"child_workflow_only,omitempty"` + // Headers that are passed by the workflow that is sending a signal to the external + // workflow that is receiving this signal. + Header *v1.Header `protobuf:"bytes,7,opt,name=header,proto3" json:"header,omitempty"` +} + +func (x *SignalExternalWorkflowExecutionCommandAttributes) Reset() { + *x = SignalExternalWorkflowExecutionCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SignalExternalWorkflowExecutionCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SignalExternalWorkflowExecutionCommandAttributes) ProtoMessage() {} + +func (x *SignalExternalWorkflowExecutionCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SignalExternalWorkflowExecutionCommandAttributes.ProtoReflect.Descriptor instead. +func (*SignalExternalWorkflowExecutionCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{8} +} + +func (x *SignalExternalWorkflowExecutionCommandAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *SignalExternalWorkflowExecutionCommandAttributes) GetExecution() *v1.WorkflowExecution { + if x != nil { + return x.Execution + } + return nil +} + +func (x *SignalExternalWorkflowExecutionCommandAttributes) GetSignalName() string { + if x != nil { + return x.SignalName + } + return "" +} + +func (x *SignalExternalWorkflowExecutionCommandAttributes) GetInput() *v1.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *SignalExternalWorkflowExecutionCommandAttributes) GetControl() string { + if x != nil { + return x.Control + } + return "" +} + +func (x *SignalExternalWorkflowExecutionCommandAttributes) GetChildWorkflowOnly() bool { + if x != nil { + return x.ChildWorkflowOnly + } + return false +} + +func (x *SignalExternalWorkflowExecutionCommandAttributes) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +type UpsertWorkflowSearchAttributesCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SearchAttributes *v1.SearchAttributes `protobuf:"bytes,1,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` +} + +func (x *UpsertWorkflowSearchAttributesCommandAttributes) Reset() { + *x = UpsertWorkflowSearchAttributesCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpsertWorkflowSearchAttributesCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpsertWorkflowSearchAttributesCommandAttributes) ProtoMessage() {} + +func (x *UpsertWorkflowSearchAttributesCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpsertWorkflowSearchAttributesCommandAttributes.ProtoReflect.Descriptor instead. +func (*UpsertWorkflowSearchAttributesCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{9} +} + +func (x *UpsertWorkflowSearchAttributesCommandAttributes) GetSearchAttributes() *v1.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +type ModifyWorkflowPropertiesCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // If set, update the workflow memo with the provided values. The values will be merged with + // the existing memo. If the user wants to delete values, a default/empty Payload should be + // used as the value for the key being deleted. + UpsertedMemo *v1.Memo `protobuf:"bytes,1,opt,name=upserted_memo,json=upsertedMemo,proto3" json:"upserted_memo,omitempty"` +} + +func (x *ModifyWorkflowPropertiesCommandAttributes) Reset() { + *x = ModifyWorkflowPropertiesCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ModifyWorkflowPropertiesCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ModifyWorkflowPropertiesCommandAttributes) ProtoMessage() {} + +func (x *ModifyWorkflowPropertiesCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ModifyWorkflowPropertiesCommandAttributes.ProtoReflect.Descriptor instead. +func (*ModifyWorkflowPropertiesCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{10} +} + +func (x *ModifyWorkflowPropertiesCommandAttributes) GetUpsertedMemo() *v1.Memo { + if x != nil { + return x.UpsertedMemo + } + return nil +} + +type RecordMarkerCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + MarkerName string `protobuf:"bytes,1,opt,name=marker_name,json=markerName,proto3" json:"marker_name,omitempty"` + Details map[string]*v1.Payloads `protobuf:"bytes,2,rep,name=details,proto3" json:"details,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Header *v1.Header `protobuf:"bytes,3,opt,name=header,proto3" json:"header,omitempty"` + Failure *v12.Failure `protobuf:"bytes,4,opt,name=failure,proto3" json:"failure,omitempty"` +} + +func (x *RecordMarkerCommandAttributes) Reset() { + *x = RecordMarkerCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RecordMarkerCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RecordMarkerCommandAttributes) ProtoMessage() {} + +func (x *RecordMarkerCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RecordMarkerCommandAttributes.ProtoReflect.Descriptor instead. +func (*RecordMarkerCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{11} +} + +func (x *RecordMarkerCommandAttributes) GetMarkerName() string { + if x != nil { + return x.MarkerName + } + return "" +} + +func (x *RecordMarkerCommandAttributes) GetDetails() map[string]*v1.Payloads { + if x != nil { + return x.Details + } + return nil +} + +func (x *RecordMarkerCommandAttributes) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *RecordMarkerCommandAttributes) GetFailure() *v12.Failure { + if x != nil { + return x.Failure + } + return nil +} + +type ContinueAsNewWorkflowExecutionCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + WorkflowType *v1.WorkflowType `protobuf:"bytes,1,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + TaskQueue *v11.TaskQueue `protobuf:"bytes,2,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + Input *v1.Payloads `protobuf:"bytes,3,opt,name=input,proto3" json:"input,omitempty"` + // Timeout of a single workflow run. + WorkflowRunTimeout *durationpb.Duration `protobuf:"bytes,4,opt,name=workflow_run_timeout,json=workflowRunTimeout,proto3" json:"workflow_run_timeout,omitempty"` + // Timeout of a single workflow task. + WorkflowTaskTimeout *durationpb.Duration `protobuf:"bytes,5,opt,name=workflow_task_timeout,json=workflowTaskTimeout,proto3" json:"workflow_task_timeout,omitempty"` + // How long the workflow start will be delayed - not really a "backoff" in the traditional sense. + BackoffStartInterval *durationpb.Duration `protobuf:"bytes,6,opt,name=backoff_start_interval,json=backoffStartInterval,proto3" json:"backoff_start_interval,omitempty"` + RetryPolicy *v1.RetryPolicy `protobuf:"bytes,7,opt,name=retry_policy,json=retryPolicy,proto3" json:"retry_policy,omitempty"` + // Should be removed + Initiator v13.ContinueAsNewInitiator `protobuf:"varint,8,opt,name=initiator,proto3,enum=temporal.api.enums.v1.ContinueAsNewInitiator" json:"initiator,omitempty"` + // Should be removed + Failure *v12.Failure `protobuf:"bytes,9,opt,name=failure,proto3" json:"failure,omitempty"` + // Should be removed + LastCompletionResult *v1.Payloads `protobuf:"bytes,10,opt,name=last_completion_result,json=lastCompletionResult,proto3" json:"last_completion_result,omitempty"` + // Should be removed. Not necessarily unused but unclear and not exposed by SDKs. + CronSchedule string `protobuf:"bytes,11,opt,name=cron_schedule,json=cronSchedule,proto3" json:"cron_schedule,omitempty"` + Header *v1.Header `protobuf:"bytes,12,opt,name=header,proto3" json:"header,omitempty"` + Memo *v1.Memo `protobuf:"bytes,13,opt,name=memo,proto3" json:"memo,omitempty"` + SearchAttributes *v1.SearchAttributes `protobuf:"bytes,14,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` + // If this is set, the new execution inherits the Build ID of the current execution. Otherwise, + // the assignment rules will be used to independently assign a Build ID to the new execution. + InheritBuildId bool `protobuf:"varint,15,opt,name=inherit_build_id,json=inheritBuildId,proto3" json:"inherit_build_id,omitempty"` +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) Reset() { + *x = ContinueAsNewWorkflowExecutionCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ContinueAsNewWorkflowExecutionCommandAttributes) ProtoMessage() {} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ContinueAsNewWorkflowExecutionCommandAttributes.ProtoReflect.Descriptor instead. +func (*ContinueAsNewWorkflowExecutionCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{12} +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetTaskQueue() *v11.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetInput() *v1.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetWorkflowRunTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowRunTimeout + } + return nil +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetWorkflowTaskTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowTaskTimeout + } + return nil +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetBackoffStartInterval() *durationpb.Duration { + if x != nil { + return x.BackoffStartInterval + } + return nil +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetRetryPolicy() *v1.RetryPolicy { + if x != nil { + return x.RetryPolicy + } + return nil +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetInitiator() v13.ContinueAsNewInitiator { + if x != nil { + return x.Initiator + } + return v13.ContinueAsNewInitiator(0) +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetFailure() *v12.Failure { + if x != nil { + return x.Failure + } + return nil +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetLastCompletionResult() *v1.Payloads { + if x != nil { + return x.LastCompletionResult + } + return nil +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetCronSchedule() string { + if x != nil { + return x.CronSchedule + } + return "" +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetMemo() *v1.Memo { + if x != nil { + return x.Memo + } + return nil +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetSearchAttributes() *v1.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +func (x *ContinueAsNewWorkflowExecutionCommandAttributes) GetInheritBuildId() bool { + if x != nil { + return x.InheritBuildId + } + return false +} + +type StartChildWorkflowExecutionCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + WorkflowId string `protobuf:"bytes,2,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + WorkflowType *v1.WorkflowType `protobuf:"bytes,3,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + TaskQueue *v11.TaskQueue `protobuf:"bytes,4,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + Input *v1.Payloads `protobuf:"bytes,5,opt,name=input,proto3" json:"input,omitempty"` + // Total workflow execution timeout including retries and continue as new. + WorkflowExecutionTimeout *durationpb.Duration `protobuf:"bytes,6,opt,name=workflow_execution_timeout,json=workflowExecutionTimeout,proto3" json:"workflow_execution_timeout,omitempty"` + // Timeout of a single workflow run. + WorkflowRunTimeout *durationpb.Duration `protobuf:"bytes,7,opt,name=workflow_run_timeout,json=workflowRunTimeout,proto3" json:"workflow_run_timeout,omitempty"` + // Timeout of a single workflow task. + WorkflowTaskTimeout *durationpb.Duration `protobuf:"bytes,8,opt,name=workflow_task_timeout,json=workflowTaskTimeout,proto3" json:"workflow_task_timeout,omitempty"` + // Default: PARENT_CLOSE_POLICY_TERMINATE. + ParentClosePolicy v13.ParentClosePolicy `protobuf:"varint,9,opt,name=parent_close_policy,json=parentClosePolicy,proto3,enum=temporal.api.enums.v1.ParentClosePolicy" json:"parent_close_policy,omitempty"` + Control string `protobuf:"bytes,10,opt,name=control,proto3" json:"control,omitempty"` + // Default: WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE. + WorkflowIdReusePolicy v13.WorkflowIdReusePolicy `protobuf:"varint,11,opt,name=workflow_id_reuse_policy,json=workflowIdReusePolicy,proto3,enum=temporal.api.enums.v1.WorkflowIdReusePolicy" json:"workflow_id_reuse_policy,omitempty"` + RetryPolicy *v1.RetryPolicy `protobuf:"bytes,12,opt,name=retry_policy,json=retryPolicy,proto3" json:"retry_policy,omitempty"` + // Establish a cron schedule for the child workflow. + CronSchedule string `protobuf:"bytes,13,opt,name=cron_schedule,json=cronSchedule,proto3" json:"cron_schedule,omitempty"` + Header *v1.Header `protobuf:"bytes,14,opt,name=header,proto3" json:"header,omitempty"` + Memo *v1.Memo `protobuf:"bytes,15,opt,name=memo,proto3" json:"memo,omitempty"` + SearchAttributes *v1.SearchAttributes `protobuf:"bytes,16,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` + // If this is set, the child workflow inherits the Build ID of the parent. Otherwise, the assignment + // rules of the child's Task Queue will be used to independently assign a Build ID to it. + InheritBuildId bool `protobuf:"varint,17,opt,name=inherit_build_id,json=inheritBuildId,proto3" json:"inherit_build_id,omitempty"` +} + +func (x *StartChildWorkflowExecutionCommandAttributes) Reset() { + *x = StartChildWorkflowExecutionCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartChildWorkflowExecutionCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartChildWorkflowExecutionCommandAttributes) ProtoMessage() {} + +func (x *StartChildWorkflowExecutionCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartChildWorkflowExecutionCommandAttributes.ProtoReflect.Descriptor instead. +func (*StartChildWorkflowExecutionCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{13} +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetTaskQueue() *v11.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetInput() *v1.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetWorkflowExecutionTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowExecutionTimeout + } + return nil +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetWorkflowRunTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowRunTimeout + } + return nil +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetWorkflowTaskTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowTaskTimeout + } + return nil +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetParentClosePolicy() v13.ParentClosePolicy { + if x != nil { + return x.ParentClosePolicy + } + return v13.ParentClosePolicy(0) +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetControl() string { + if x != nil { + return x.Control + } + return "" +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetWorkflowIdReusePolicy() v13.WorkflowIdReusePolicy { + if x != nil { + return x.WorkflowIdReusePolicy + } + return v13.WorkflowIdReusePolicy(0) +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetRetryPolicy() *v1.RetryPolicy { + if x != nil { + return x.RetryPolicy + } + return nil +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetCronSchedule() string { + if x != nil { + return x.CronSchedule + } + return "" +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetMemo() *v1.Memo { + if x != nil { + return x.Memo + } + return nil +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetSearchAttributes() *v1.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +func (x *StartChildWorkflowExecutionCommandAttributes) GetInheritBuildId() bool { + if x != nil { + return x.InheritBuildId + } + return false +} + +type ProtocolMessageCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The message ID of the message to which this command is a pointer. + MessageId string `protobuf:"bytes,1,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"` +} + +func (x *ProtocolMessageCommandAttributes) Reset() { + *x = ProtocolMessageCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ProtocolMessageCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProtocolMessageCommandAttributes) ProtoMessage() {} + +func (x *ProtocolMessageCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ProtocolMessageCommandAttributes.ProtoReflect.Descriptor instead. +func (*ProtocolMessageCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{14} +} + +func (x *ProtocolMessageCommandAttributes) GetMessageId() string { + if x != nil { + return x.MessageId + } + return "" +} + +type ScheduleNexusOperationCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Endpoint name, must exist in the endpoint registry or this command will fail. + Endpoint string `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + // Service name. + Service string `protobuf:"bytes,2,opt,name=service,proto3" json:"service,omitempty"` + // Operation name. + Operation string `protobuf:"bytes,3,opt,name=operation,proto3" json:"operation,omitempty"` + // Input for the operation. The server converts this into Nexus request content and the appropriate content headers + // internally when sending the StartOperation request. On the handler side, if it is also backed by Temporal, the + // content is transformed back to the original Payload sent in this command. + Input *v1.Payload `protobuf:"bytes,4,opt,name=input,proto3" json:"input,omitempty"` + // Schedule-to-close timeout for this operation. + // Indicates how long the caller is willing to wait for operation completion. + // Calls are retried internally by the server. + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + ScheduleToCloseTimeout *durationpb.Duration `protobuf:"bytes,5,opt,name=schedule_to_close_timeout,json=scheduleToCloseTimeout,proto3" json:"schedule_to_close_timeout,omitempty"` + // Header to attach to the Nexus request. + // Users are responsible for encrypting sensitive data in this header as it is stored in workflow history and + // transmitted to external services as-is. + // This is useful for propagating tracing information. + // Note these headers are not the same as Temporal headers on internal activities and child workflows, these are + // transmitted to Nexus operations that may be external and are not traditional payloads. + NexusHeader map[string]string `protobuf:"bytes,6,rep,name=nexus_header,json=nexusHeader,proto3" json:"nexus_header,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *ScheduleNexusOperationCommandAttributes) Reset() { + *x = ScheduleNexusOperationCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ScheduleNexusOperationCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ScheduleNexusOperationCommandAttributes) ProtoMessage() {} + +func (x *ScheduleNexusOperationCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[15] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ScheduleNexusOperationCommandAttributes.ProtoReflect.Descriptor instead. +func (*ScheduleNexusOperationCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{15} +} + +func (x *ScheduleNexusOperationCommandAttributes) GetEndpoint() string { + if x != nil { + return x.Endpoint + } + return "" +} + +func (x *ScheduleNexusOperationCommandAttributes) GetService() string { + if x != nil { + return x.Service + } + return "" +} + +func (x *ScheduleNexusOperationCommandAttributes) GetOperation() string { + if x != nil { + return x.Operation + } + return "" +} + +func (x *ScheduleNexusOperationCommandAttributes) GetInput() *v1.Payload { + if x != nil { + return x.Input + } + return nil +} + +func (x *ScheduleNexusOperationCommandAttributes) GetScheduleToCloseTimeout() *durationpb.Duration { + if x != nil { + return x.ScheduleToCloseTimeout + } + return nil +} + +func (x *ScheduleNexusOperationCommandAttributes) GetNexusHeader() map[string]string { + if x != nil { + return x.NexusHeader + } + return nil +} + +type RequestCancelNexusOperationCommandAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The `NEXUS_OPERATION_SCHEDULED` event ID (a unique identifier) for the operation to be canceled. + // The operation may ignore cancellation and end up with any completion state. + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` +} + +func (x *RequestCancelNexusOperationCommandAttributes) Reset() { + *x = RequestCancelNexusOperationCommandAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RequestCancelNexusOperationCommandAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RequestCancelNexusOperationCommandAttributes) ProtoMessage() {} + +func (x *RequestCancelNexusOperationCommandAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[16] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RequestCancelNexusOperationCommandAttributes.ProtoReflect.Descriptor instead. +func (*RequestCancelNexusOperationCommandAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{16} +} + +func (x *RequestCancelNexusOperationCommandAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +type Command struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CommandType v13.CommandType `protobuf:"varint,1,opt,name=command_type,json=commandType,proto3,enum=temporal.api.enums.v1.CommandType" json:"command_type,omitempty"` + // Metadata on the command. This is sometimes carried over to the history event if one is + // created as a result of the command. Most commands won't have this information, and how this + // information is used is dependent upon the interface that reads it. + // + // Current well-known uses: + // - start_child_workflow_execution_command_attributes - populates + // temporal.api.workflow.v1.WorkflowExecutionInfo.user_metadata where the summary and details + // are used by user interfaces to show fixed as-of-start workflow summary and details. + // - start_timer_command_attributes - populates temporal.api.history.v1.HistoryEvent for timer + // started where the summary is used to identify the timer. + UserMetadata *v14.UserMetadata `protobuf:"bytes,301,opt,name=user_metadata,json=userMetadata,proto3" json:"user_metadata,omitempty"` + // The command details. The type must match that in `command_type`. + // + // Types that are assignable to Attributes: + // + // *Command_ScheduleActivityTaskCommandAttributes + // *Command_StartTimerCommandAttributes + // *Command_CompleteWorkflowExecutionCommandAttributes + // *Command_FailWorkflowExecutionCommandAttributes + // *Command_RequestCancelActivityTaskCommandAttributes + // *Command_CancelTimerCommandAttributes + // *Command_CancelWorkflowExecutionCommandAttributes + // *Command_RequestCancelExternalWorkflowExecutionCommandAttributes + // *Command_RecordMarkerCommandAttributes + // *Command_ContinueAsNewWorkflowExecutionCommandAttributes + // *Command_StartChildWorkflowExecutionCommandAttributes + // *Command_SignalExternalWorkflowExecutionCommandAttributes + // *Command_UpsertWorkflowSearchAttributesCommandAttributes + // *Command_ProtocolMessageCommandAttributes + // *Command_ModifyWorkflowPropertiesCommandAttributes + // *Command_ScheduleNexusOperationCommandAttributes + // *Command_RequestCancelNexusOperationCommandAttributes + Attributes isCommand_Attributes `protobuf_oneof:"attributes"` +} + +func (x *Command) Reset() { + *x = Command{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Command) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Command) ProtoMessage() {} + +func (x *Command) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_command_v1_message_proto_msgTypes[17] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Command.ProtoReflect.Descriptor instead. +func (*Command) Descriptor() ([]byte, []int) { + return file_temporal_api_command_v1_message_proto_rawDescGZIP(), []int{17} +} + +func (x *Command) GetCommandType() v13.CommandType { + if x != nil { + return x.CommandType + } + return v13.CommandType(0) +} + +func (x *Command) GetUserMetadata() *v14.UserMetadata { + if x != nil { + return x.UserMetadata + } + return nil +} + +func (m *Command) GetAttributes() isCommand_Attributes { + if m != nil { + return m.Attributes + } + return nil +} + +func (x *Command) GetScheduleActivityTaskCommandAttributes() *ScheduleActivityTaskCommandAttributes { + if x, ok := x.GetAttributes().(*Command_ScheduleActivityTaskCommandAttributes); ok { + return x.ScheduleActivityTaskCommandAttributes + } + return nil +} + +func (x *Command) GetStartTimerCommandAttributes() *StartTimerCommandAttributes { + if x, ok := x.GetAttributes().(*Command_StartTimerCommandAttributes); ok { + return x.StartTimerCommandAttributes + } + return nil +} + +func (x *Command) GetCompleteWorkflowExecutionCommandAttributes() *CompleteWorkflowExecutionCommandAttributes { + if x, ok := x.GetAttributes().(*Command_CompleteWorkflowExecutionCommandAttributes); ok { + return x.CompleteWorkflowExecutionCommandAttributes + } + return nil +} + +func (x *Command) GetFailWorkflowExecutionCommandAttributes() *FailWorkflowExecutionCommandAttributes { + if x, ok := x.GetAttributes().(*Command_FailWorkflowExecutionCommandAttributes); ok { + return x.FailWorkflowExecutionCommandAttributes + } + return nil +} + +func (x *Command) GetRequestCancelActivityTaskCommandAttributes() *RequestCancelActivityTaskCommandAttributes { + if x, ok := x.GetAttributes().(*Command_RequestCancelActivityTaskCommandAttributes); ok { + return x.RequestCancelActivityTaskCommandAttributes + } + return nil +} + +func (x *Command) GetCancelTimerCommandAttributes() *CancelTimerCommandAttributes { + if x, ok := x.GetAttributes().(*Command_CancelTimerCommandAttributes); ok { + return x.CancelTimerCommandAttributes + } + return nil +} + +func (x *Command) GetCancelWorkflowExecutionCommandAttributes() *CancelWorkflowExecutionCommandAttributes { + if x, ok := x.GetAttributes().(*Command_CancelWorkflowExecutionCommandAttributes); ok { + return x.CancelWorkflowExecutionCommandAttributes + } + return nil +} + +func (x *Command) GetRequestCancelExternalWorkflowExecutionCommandAttributes() *RequestCancelExternalWorkflowExecutionCommandAttributes { + if x, ok := x.GetAttributes().(*Command_RequestCancelExternalWorkflowExecutionCommandAttributes); ok { + return x.RequestCancelExternalWorkflowExecutionCommandAttributes + } + return nil +} + +func (x *Command) GetRecordMarkerCommandAttributes() *RecordMarkerCommandAttributes { + if x, ok := x.GetAttributes().(*Command_RecordMarkerCommandAttributes); ok { + return x.RecordMarkerCommandAttributes + } + return nil +} + +func (x *Command) GetContinueAsNewWorkflowExecutionCommandAttributes() *ContinueAsNewWorkflowExecutionCommandAttributes { + if x, ok := x.GetAttributes().(*Command_ContinueAsNewWorkflowExecutionCommandAttributes); ok { + return x.ContinueAsNewWorkflowExecutionCommandAttributes + } + return nil +} + +func (x *Command) GetStartChildWorkflowExecutionCommandAttributes() *StartChildWorkflowExecutionCommandAttributes { + if x, ok := x.GetAttributes().(*Command_StartChildWorkflowExecutionCommandAttributes); ok { + return x.StartChildWorkflowExecutionCommandAttributes + } + return nil +} + +func (x *Command) GetSignalExternalWorkflowExecutionCommandAttributes() *SignalExternalWorkflowExecutionCommandAttributes { + if x, ok := x.GetAttributes().(*Command_SignalExternalWorkflowExecutionCommandAttributes); ok { + return x.SignalExternalWorkflowExecutionCommandAttributes + } + return nil +} + +func (x *Command) GetUpsertWorkflowSearchAttributesCommandAttributes() *UpsertWorkflowSearchAttributesCommandAttributes { + if x, ok := x.GetAttributes().(*Command_UpsertWorkflowSearchAttributesCommandAttributes); ok { + return x.UpsertWorkflowSearchAttributesCommandAttributes + } + return nil +} + +func (x *Command) GetProtocolMessageCommandAttributes() *ProtocolMessageCommandAttributes { + if x, ok := x.GetAttributes().(*Command_ProtocolMessageCommandAttributes); ok { + return x.ProtocolMessageCommandAttributes + } + return nil +} + +func (x *Command) GetModifyWorkflowPropertiesCommandAttributes() *ModifyWorkflowPropertiesCommandAttributes { + if x, ok := x.GetAttributes().(*Command_ModifyWorkflowPropertiesCommandAttributes); ok { + return x.ModifyWorkflowPropertiesCommandAttributes + } + return nil +} + +func (x *Command) GetScheduleNexusOperationCommandAttributes() *ScheduleNexusOperationCommandAttributes { + if x, ok := x.GetAttributes().(*Command_ScheduleNexusOperationCommandAttributes); ok { + return x.ScheduleNexusOperationCommandAttributes + } + return nil +} + +func (x *Command) GetRequestCancelNexusOperationCommandAttributes() *RequestCancelNexusOperationCommandAttributes { + if x, ok := x.GetAttributes().(*Command_RequestCancelNexusOperationCommandAttributes); ok { + return x.RequestCancelNexusOperationCommandAttributes + } + return nil +} + +type isCommand_Attributes interface { + isCommand_Attributes() +} + +type Command_ScheduleActivityTaskCommandAttributes struct { + ScheduleActivityTaskCommandAttributes *ScheduleActivityTaskCommandAttributes `protobuf:"bytes,2,opt,name=schedule_activity_task_command_attributes,json=scheduleActivityTaskCommandAttributes,proto3,oneof"` +} + +type Command_StartTimerCommandAttributes struct { + StartTimerCommandAttributes *StartTimerCommandAttributes `protobuf:"bytes,3,opt,name=start_timer_command_attributes,json=startTimerCommandAttributes,proto3,oneof"` +} + +type Command_CompleteWorkflowExecutionCommandAttributes struct { + CompleteWorkflowExecutionCommandAttributes *CompleteWorkflowExecutionCommandAttributes `protobuf:"bytes,4,opt,name=complete_workflow_execution_command_attributes,json=completeWorkflowExecutionCommandAttributes,proto3,oneof"` +} + +type Command_FailWorkflowExecutionCommandAttributes struct { + FailWorkflowExecutionCommandAttributes *FailWorkflowExecutionCommandAttributes `protobuf:"bytes,5,opt,name=fail_workflow_execution_command_attributes,json=failWorkflowExecutionCommandAttributes,proto3,oneof"` +} + +type Command_RequestCancelActivityTaskCommandAttributes struct { + RequestCancelActivityTaskCommandAttributes *RequestCancelActivityTaskCommandAttributes `protobuf:"bytes,6,opt,name=request_cancel_activity_task_command_attributes,json=requestCancelActivityTaskCommandAttributes,proto3,oneof"` +} + +type Command_CancelTimerCommandAttributes struct { + CancelTimerCommandAttributes *CancelTimerCommandAttributes `protobuf:"bytes,7,opt,name=cancel_timer_command_attributes,json=cancelTimerCommandAttributes,proto3,oneof"` +} + +type Command_CancelWorkflowExecutionCommandAttributes struct { + CancelWorkflowExecutionCommandAttributes *CancelWorkflowExecutionCommandAttributes `protobuf:"bytes,8,opt,name=cancel_workflow_execution_command_attributes,json=cancelWorkflowExecutionCommandAttributes,proto3,oneof"` +} + +type Command_RequestCancelExternalWorkflowExecutionCommandAttributes struct { + RequestCancelExternalWorkflowExecutionCommandAttributes *RequestCancelExternalWorkflowExecutionCommandAttributes `protobuf:"bytes,9,opt,name=request_cancel_external_workflow_execution_command_attributes,json=requestCancelExternalWorkflowExecutionCommandAttributes,proto3,oneof"` +} + +type Command_RecordMarkerCommandAttributes struct { + RecordMarkerCommandAttributes *RecordMarkerCommandAttributes `protobuf:"bytes,10,opt,name=record_marker_command_attributes,json=recordMarkerCommandAttributes,proto3,oneof"` +} + +type Command_ContinueAsNewWorkflowExecutionCommandAttributes struct { + ContinueAsNewWorkflowExecutionCommandAttributes *ContinueAsNewWorkflowExecutionCommandAttributes `protobuf:"bytes,11,opt,name=continue_as_new_workflow_execution_command_attributes,json=continueAsNewWorkflowExecutionCommandAttributes,proto3,oneof"` +} + +type Command_StartChildWorkflowExecutionCommandAttributes struct { + StartChildWorkflowExecutionCommandAttributes *StartChildWorkflowExecutionCommandAttributes `protobuf:"bytes,12,opt,name=start_child_workflow_execution_command_attributes,json=startChildWorkflowExecutionCommandAttributes,proto3,oneof"` +} + +type Command_SignalExternalWorkflowExecutionCommandAttributes struct { + SignalExternalWorkflowExecutionCommandAttributes *SignalExternalWorkflowExecutionCommandAttributes `protobuf:"bytes,13,opt,name=signal_external_workflow_execution_command_attributes,json=signalExternalWorkflowExecutionCommandAttributes,proto3,oneof"` +} + +type Command_UpsertWorkflowSearchAttributesCommandAttributes struct { + UpsertWorkflowSearchAttributesCommandAttributes *UpsertWorkflowSearchAttributesCommandAttributes `protobuf:"bytes,14,opt,name=upsert_workflow_search_attributes_command_attributes,json=upsertWorkflowSearchAttributesCommandAttributes,proto3,oneof"` +} + +type Command_ProtocolMessageCommandAttributes struct { + ProtocolMessageCommandAttributes *ProtocolMessageCommandAttributes `protobuf:"bytes,15,opt,name=protocol_message_command_attributes,json=protocolMessageCommandAttributes,proto3,oneof"` +} + +type Command_ModifyWorkflowPropertiesCommandAttributes struct { + // 16 is available for use - it was used as part of a prototype that never made it into a release + ModifyWorkflowPropertiesCommandAttributes *ModifyWorkflowPropertiesCommandAttributes `protobuf:"bytes,17,opt,name=modify_workflow_properties_command_attributes,json=modifyWorkflowPropertiesCommandAttributes,proto3,oneof"` +} + +type Command_ScheduleNexusOperationCommandAttributes struct { + ScheduleNexusOperationCommandAttributes *ScheduleNexusOperationCommandAttributes `protobuf:"bytes,18,opt,name=schedule_nexus_operation_command_attributes,json=scheduleNexusOperationCommandAttributes,proto3,oneof"` +} + +type Command_RequestCancelNexusOperationCommandAttributes struct { + RequestCancelNexusOperationCommandAttributes *RequestCancelNexusOperationCommandAttributes `protobuf:"bytes,19,opt,name=request_cancel_nexus_operation_command_attributes,json=requestCancelNexusOperationCommandAttributes,proto3,oneof"` +} + +func (*Command_ScheduleActivityTaskCommandAttributes) isCommand_Attributes() {} + +func (*Command_StartTimerCommandAttributes) isCommand_Attributes() {} + +func (*Command_CompleteWorkflowExecutionCommandAttributes) isCommand_Attributes() {} + +func (*Command_FailWorkflowExecutionCommandAttributes) isCommand_Attributes() {} + +func (*Command_RequestCancelActivityTaskCommandAttributes) isCommand_Attributes() {} + +func (*Command_CancelTimerCommandAttributes) isCommand_Attributes() {} + +func (*Command_CancelWorkflowExecutionCommandAttributes) isCommand_Attributes() {} + +func (*Command_RequestCancelExternalWorkflowExecutionCommandAttributes) isCommand_Attributes() {} + +func (*Command_RecordMarkerCommandAttributes) isCommand_Attributes() {} + +func (*Command_ContinueAsNewWorkflowExecutionCommandAttributes) isCommand_Attributes() {} + +func (*Command_StartChildWorkflowExecutionCommandAttributes) isCommand_Attributes() {} + +func (*Command_SignalExternalWorkflowExecutionCommandAttributes) isCommand_Attributes() {} + +func (*Command_UpsertWorkflowSearchAttributesCommandAttributes) isCommand_Attributes() {} + +func (*Command_ProtocolMessageCommandAttributes) isCommand_Attributes() {} + +func (*Command_ModifyWorkflowPropertiesCommandAttributes) isCommand_Attributes() {} + +func (*Command_ScheduleNexusOperationCommandAttributes) isCommand_Attributes() {} + +func (*Command_RequestCancelNexusOperationCommandAttributes) isCommand_Attributes() {} + +var File_temporal_api_command_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_command_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x25, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x17, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x1a, 0x1e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x75, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, + 0x31, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x1a, 0x28, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, + 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x74, 0x79, + 0x70, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x1a, 0x27, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, + 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x73, 0x64, 0x6b, 0x2f, 0x76, 0x31, 0x2f, 0x75, 0x73, + 0x65, 0x72, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x22, 0xf5, 0x06, 0x0a, 0x25, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x41, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0b, 0x61, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x61, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x61, + 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, + 0x74, 0x79, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, + 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, + 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, + 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, + 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, + 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x58, 0x0a, 0x19, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x63, + 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x16, 0x73, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x6f, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x58, 0x0a, 0x19, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x16, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x6f, 0x53, 0x74, 0x61, + 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x52, 0x0a, 0x16, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x6f, 0x43, 0x6c, 0x6f, + 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x11, + 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, + 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x10, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, + 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x0c, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x70, + 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0b, 0x72, + 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, + 0x0a, 0x17, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x65, 0x61, 0x67, 0x65, 0x72, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x45, 0x61, 0x67, 0x65, 0x72, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x35, 0x0a, 0x15, 0x75, 0x73, 0x65, 0x5f, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, + 0x0d, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x75, 0x73, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x4a, 0x04, 0x08, 0x03, + 0x10, 0x04, 0x22, 0x5e, 0x0a, 0x2a, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, + 0x65, 0x6c, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, + 0x12, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, + 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8e, 0x01, + 0x0a, 0x1b, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x1d, 0x0a, 0x08, + 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x74, + 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x50, 0x0a, 0x15, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x5f, 0x74, 0x6f, 0x5f, 0x66, 0x69, 0x72, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x12, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x6f, 0x46, 0x69, 0x72, 0x65, 0x54, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0x6a, 0x0a, 0x2a, 0x43, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x73, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, + 0x68, 0x0a, 0x26, 0x46, 0x61, 0x69, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x3e, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x3d, 0x0a, 0x1c, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x54, 0x69, + 0x6d, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x12, 0x1d, 0x0a, 0x08, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x6a, 0x0a, 0x28, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x3e, 0x0a, 0x07, 0x64, + 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x07, 0x64, 0x65, + 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x89, 0x02, 0x0a, 0x37, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x19, 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x72, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x1c, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x32, 0x0a, 0x13, + 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6f, 0x6e, + 0x6c, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x4f, 0x6e, 0x6c, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1a, + 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, + 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x90, 0x03, 0x0a, 0x30, 0x53, 0x69, + 0x67, 0x6e, 0x61, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x20, 0x0a, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x09, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x69, 0x67, 0x6e, + 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x69, 0x6e, 0x70, + 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x32, 0x0a, 0x13, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x63, 0x68, 0x69, + 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x4f, 0x6e, 0x6c, 0x79, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, + 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8c, 0x01, 0x0a, 0x2f, 0x55, + 0x70, 0x73, 0x65, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x61, 0x72, + 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x59, 0x0a, 0x11, + 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, + 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x10, + 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x72, 0x0a, 0x29, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x45, + 0x0a, 0x0d, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x6d, + 0x6f, 0x52, 0x0c, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x6d, 0x6f, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x89, 0x03, 0x0a, 0x1d, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x4d, 0x61, 0x72, + 0x6b, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0b, 0x6d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x4e, 0x61, + 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x61, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x2e, + 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x4d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x2e, 0x44, 0x65, 0x74, + 0x61, 0x69, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, + 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, + 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, + 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x64, 0x0a, 0x0c, 0x44, + 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x3a, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xdb, 0x08, + 0x0a, 0x2f, 0x43, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x41, 0x73, 0x4e, 0x65, 0x77, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, + 0x4d, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0a, 0x74, 0x61, + 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, + 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, + 0x75, 0x65, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x3a, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, + 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, 0x14, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, + 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x54, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x51, 0x0a, 0x15, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x13, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x54, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x53, 0x0a, 0x16, 0x62, 0x61, 0x63, 0x6b, 0x6f, 0x66, + 0x66, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x14, 0x62, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x53, 0x74, 0x61, 0x72, 0x74, 0x49, 0x6e, 0x74, 0x65, + 0x72, 0x76, 0x61, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x0c, 0x72, 0x65, 0x74, 0x72, + 0x79, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x52, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x4f, 0x0a, 0x09, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x2d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x69, 0x6e, + 0x75, 0x65, 0x41, 0x73, 0x4e, 0x65, 0x77, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x6f, 0x72, + 0x52, 0x09, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x6f, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, + 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, + 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, + 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5a, 0x0a, 0x16, 0x6c, + 0x61, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x14, 0x6c, 0x61, 0x73, 0x74, + 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x27, 0x0a, 0x0d, 0x63, 0x72, 0x6f, 0x6e, 0x5f, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x72, 0x6f, 0x6e, 0x53, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x68, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x18, 0x0d, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x6d, 0x6f, 0x52, 0x04, + 0x6d, 0x65, 0x6d, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, 0x11, 0x73, 0x65, 0x61, 0x72, 0x63, + 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x0e, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x10, 0x73, 0x65, 0x61, 0x72, 0x63, + 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x2c, 0x0a, 0x10, 0x69, 0x6e, 0x68, 0x65, 0x72, 0x69, 0x74, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, + 0x69, 0x64, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x68, 0x65, 0x72, 0x69, 0x74, + 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xa1, 0x09, 0x0a, 0x2c, 0x53, + 0x74, 0x61, 0x72, 0x74, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, + 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0a, 0x74, 0x61, 0x73, + 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, + 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, + 0x65, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x3a, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x05, + 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5b, 0x0a, 0x1a, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x18, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x4f, 0x0a, 0x14, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x72, 0x75, + 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x52, 0x75, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x51, 0x0a, 0x15, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x54, 0x61, 0x73, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, + 0x0a, 0x13, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x70, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x52, 0x11, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, + 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, + 0x72, 0x6f, 0x6c, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, + 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x69, 0x0a, 0x18, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x69, 0x64, 0x5f, 0x72, 0x65, 0x75, 0x73, 0x65, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x52, 0x65, 0x75, 0x73, 0x65, 0x50, 0x6f, 0x6c, 0x69, 0x63, + 0x79, 0x52, 0x15, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x52, 0x65, 0x75, + 0x73, 0x65, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x0c, 0x72, + 0x65, 0x74, 0x72, 0x79, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x23, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, 0x0a, 0x0d, 0x63, 0x72, 0x6f, 0x6e, 0x5f, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x72, 0x6f, 0x6e, + 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x68, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x18, 0x0f, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x6d, 0x6f, 0x52, + 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, 0x11, 0x73, 0x65, 0x61, 0x72, + 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x10, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, + 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x10, 0x73, 0x65, 0x61, 0x72, + 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x2c, 0x0a, 0x10, 0x69, 0x6e, 0x68, 0x65, 0x72, 0x69, 0x74, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, + 0x5f, 0x69, 0x64, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x68, 0x65, 0x72, 0x69, + 0x74, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x45, 0x0a, 0x20, 0x50, + 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x21, + 0x0a, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, + 0xe0, 0x03, 0x0a, 0x27, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x1e, 0x0a, 0x08, 0x65, 0x6e, 0x64, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x39, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x58, 0x0a, + 0x19, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6c, 0x6f, + 0x73, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x16, 0x73, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x54, 0x6f, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x78, 0x0a, 0x0c, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x5f, 0x68, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x51, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, + 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x2e, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x48, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x48, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x46, 0x0a, 0x10, 0x4e, 0x65, 0x78, 0x75, + 0x73, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x18, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x60, 0x0a, + 0x2c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4e, 0x65, 0x78, + 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, 0x73, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xfd, 0x17, 0x0a, 0x07, + 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x49, 0x0a, 0x0c, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x22, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, + 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, + 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, + 0x0a, 0x0d, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, + 0xad, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x4d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x0c, 0x75, 0x73, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, 0x9e, 0x01, 0x0a, 0x29, 0x73, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x61, + 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x41, 0x63, 0x74, 0x69, 0x76, + 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x25, 0x73, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x7f, 0x0a, 0x1e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x72, + 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, + 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x1b, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xad, 0x01, + 0x0a, 0x2e, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, + 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2a, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, + 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xa1, 0x01, 0x0a, 0x2a, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, + 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x26, 0x66, 0x61, 0x69, 0x6c, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0xae, 0x01, 0x0a, 0x2f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x63, + 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x61, + 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, + 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2a, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x41, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x82, 0x01, 0x0a, + 0x1f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, + 0x6e, 0x63, 0x65, 0x6c, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x1c, 0x63, 0x61, 0x6e, + 0x63, 0x65, 0x6c, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xa7, 0x01, 0x0a, 0x2c, + 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x41, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x48, 0x00, 0x52, 0x28, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xd6, + 0x01, 0x0a, 0x3d, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, + 0x5f, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x09, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x50, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x48, 0x00, 0x52, 0x37, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, + 0x65, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x85, 0x01, + 0x0a, 0x20, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x6d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x5f, + 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, + 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x4d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x1d, + 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x4d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x12, 0xbe, 0x01, 0x0a, 0x35, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x5f, 0x61, 0x73, 0x5f, + 0x6e, 0x65, 0x77, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x48, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x41, 0x73, + 0x4e, 0x65, 0x77, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2f, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x41, + 0x73, 0x4e, 0x65, 0x77, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xb4, 0x01, 0x0a, 0x31, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x5f, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x0c, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, + 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2c, 0x73, 0x74, 0x61, 0x72, 0x74, 0x43, 0x68, + 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xc0, 0x01, 0x0a, 0x35, 0x73, 0x69, 0x67, 0x6e, + 0x61, 0x6c, 0x5f, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, + 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x49, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, + 0x67, 0x6e, 0x61, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x30, + 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0xbd, 0x01, 0x0a, 0x34, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x61, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x48, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x73, 0x65, 0x72, 0x74, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2f, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x8e, 0x01, 0x0a, 0x23, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x63, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x2e, 0x50, + 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, + 0x52, 0x20, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xaa, 0x01, 0x0a, 0x2d, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x5f, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, + 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x29, 0x6d, + 0x6f, 0x64, 0x69, 0x66, 0x79, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x70, + 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xa4, 0x01, 0x0a, 0x2b, 0x73, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x5f, 0x6f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x40, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4e, + 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, + 0x27, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xb4, 0x01, 0x0a, 0x31, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x5f, 0x6e, 0x65, 0x78, + 0x75, 0x73, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x13, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2c, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x42, 0x0c, 0x0a, 0x0a, 0x61, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x8e, 0x01, 0x0a, 0x1a, 0x69, 0x6f, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, + 0x6f, 0x50, 0x01, 0x5a, 0x25, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2f, 0x76, + 0x31, 0x3b, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0xaa, 0x02, 0x19, 0x54, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1c, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, + 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x3a, 0x3a, 0x56, + 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_command_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_command_v1_message_proto_rawDescData = file_temporal_api_command_v1_message_proto_rawDesc +) + +func file_temporal_api_command_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_command_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_command_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_command_v1_message_proto_rawDescData) + }) + return file_temporal_api_command_v1_message_proto_rawDescData +} + +var file_temporal_api_command_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 20) +var file_temporal_api_command_v1_message_proto_goTypes = []any{ + (*ScheduleActivityTaskCommandAttributes)(nil), // 0: temporal.api.command.v1.ScheduleActivityTaskCommandAttributes + (*RequestCancelActivityTaskCommandAttributes)(nil), // 1: temporal.api.command.v1.RequestCancelActivityTaskCommandAttributes + (*StartTimerCommandAttributes)(nil), // 2: temporal.api.command.v1.StartTimerCommandAttributes + (*CompleteWorkflowExecutionCommandAttributes)(nil), // 3: temporal.api.command.v1.CompleteWorkflowExecutionCommandAttributes + (*FailWorkflowExecutionCommandAttributes)(nil), // 4: temporal.api.command.v1.FailWorkflowExecutionCommandAttributes + (*CancelTimerCommandAttributes)(nil), // 5: temporal.api.command.v1.CancelTimerCommandAttributes + (*CancelWorkflowExecutionCommandAttributes)(nil), // 6: temporal.api.command.v1.CancelWorkflowExecutionCommandAttributes + (*RequestCancelExternalWorkflowExecutionCommandAttributes)(nil), // 7: temporal.api.command.v1.RequestCancelExternalWorkflowExecutionCommandAttributes + (*SignalExternalWorkflowExecutionCommandAttributes)(nil), // 8: temporal.api.command.v1.SignalExternalWorkflowExecutionCommandAttributes + (*UpsertWorkflowSearchAttributesCommandAttributes)(nil), // 9: temporal.api.command.v1.UpsertWorkflowSearchAttributesCommandAttributes + (*ModifyWorkflowPropertiesCommandAttributes)(nil), // 10: temporal.api.command.v1.ModifyWorkflowPropertiesCommandAttributes + (*RecordMarkerCommandAttributes)(nil), // 11: temporal.api.command.v1.RecordMarkerCommandAttributes + (*ContinueAsNewWorkflowExecutionCommandAttributes)(nil), // 12: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes + (*StartChildWorkflowExecutionCommandAttributes)(nil), // 13: temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes + (*ProtocolMessageCommandAttributes)(nil), // 14: temporal.api.command.v1.ProtocolMessageCommandAttributes + (*ScheduleNexusOperationCommandAttributes)(nil), // 15: temporal.api.command.v1.ScheduleNexusOperationCommandAttributes + (*RequestCancelNexusOperationCommandAttributes)(nil), // 16: temporal.api.command.v1.RequestCancelNexusOperationCommandAttributes + (*Command)(nil), // 17: temporal.api.command.v1.Command + nil, // 18: temporal.api.command.v1.RecordMarkerCommandAttributes.DetailsEntry + nil, // 19: temporal.api.command.v1.ScheduleNexusOperationCommandAttributes.NexusHeaderEntry + (*v1.ActivityType)(nil), // 20: temporal.api.common.v1.ActivityType + (*v11.TaskQueue)(nil), // 21: temporal.api.taskqueue.v1.TaskQueue + (*v1.Header)(nil), // 22: temporal.api.common.v1.Header + (*v1.Payloads)(nil), // 23: temporal.api.common.v1.Payloads + (*durationpb.Duration)(nil), // 24: google.protobuf.Duration + (*v1.RetryPolicy)(nil), // 25: temporal.api.common.v1.RetryPolicy + (*v12.Failure)(nil), // 26: temporal.api.failure.v1.Failure + (*v1.WorkflowExecution)(nil), // 27: temporal.api.common.v1.WorkflowExecution + (*v1.SearchAttributes)(nil), // 28: temporal.api.common.v1.SearchAttributes + (*v1.Memo)(nil), // 29: temporal.api.common.v1.Memo + (*v1.WorkflowType)(nil), // 30: temporal.api.common.v1.WorkflowType + (v13.ContinueAsNewInitiator)(0), // 31: temporal.api.enums.v1.ContinueAsNewInitiator + (v13.ParentClosePolicy)(0), // 32: temporal.api.enums.v1.ParentClosePolicy + (v13.WorkflowIdReusePolicy)(0), // 33: temporal.api.enums.v1.WorkflowIdReusePolicy + (*v1.Payload)(nil), // 34: temporal.api.common.v1.Payload + (v13.CommandType)(0), // 35: temporal.api.enums.v1.CommandType + (*v14.UserMetadata)(nil), // 36: temporal.api.sdk.v1.UserMetadata +} +var file_temporal_api_command_v1_message_proto_depIdxs = []int32{ + 20, // 0: temporal.api.command.v1.ScheduleActivityTaskCommandAttributes.activity_type:type_name -> temporal.api.common.v1.ActivityType + 21, // 1: temporal.api.command.v1.ScheduleActivityTaskCommandAttributes.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 22, // 2: temporal.api.command.v1.ScheduleActivityTaskCommandAttributes.header:type_name -> temporal.api.common.v1.Header + 23, // 3: temporal.api.command.v1.ScheduleActivityTaskCommandAttributes.input:type_name -> temporal.api.common.v1.Payloads + 24, // 4: temporal.api.command.v1.ScheduleActivityTaskCommandAttributes.schedule_to_close_timeout:type_name -> google.protobuf.Duration + 24, // 5: temporal.api.command.v1.ScheduleActivityTaskCommandAttributes.schedule_to_start_timeout:type_name -> google.protobuf.Duration + 24, // 6: temporal.api.command.v1.ScheduleActivityTaskCommandAttributes.start_to_close_timeout:type_name -> google.protobuf.Duration + 24, // 7: temporal.api.command.v1.ScheduleActivityTaskCommandAttributes.heartbeat_timeout:type_name -> google.protobuf.Duration + 25, // 8: temporal.api.command.v1.ScheduleActivityTaskCommandAttributes.retry_policy:type_name -> temporal.api.common.v1.RetryPolicy + 24, // 9: temporal.api.command.v1.StartTimerCommandAttributes.start_to_fire_timeout:type_name -> google.protobuf.Duration + 23, // 10: temporal.api.command.v1.CompleteWorkflowExecutionCommandAttributes.result:type_name -> temporal.api.common.v1.Payloads + 26, // 11: temporal.api.command.v1.FailWorkflowExecutionCommandAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 23, // 12: temporal.api.command.v1.CancelWorkflowExecutionCommandAttributes.details:type_name -> temporal.api.common.v1.Payloads + 27, // 13: temporal.api.command.v1.SignalExternalWorkflowExecutionCommandAttributes.execution:type_name -> temporal.api.common.v1.WorkflowExecution + 23, // 14: temporal.api.command.v1.SignalExternalWorkflowExecutionCommandAttributes.input:type_name -> temporal.api.common.v1.Payloads + 22, // 15: temporal.api.command.v1.SignalExternalWorkflowExecutionCommandAttributes.header:type_name -> temporal.api.common.v1.Header + 28, // 16: temporal.api.command.v1.UpsertWorkflowSearchAttributesCommandAttributes.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 29, // 17: temporal.api.command.v1.ModifyWorkflowPropertiesCommandAttributes.upserted_memo:type_name -> temporal.api.common.v1.Memo + 18, // 18: temporal.api.command.v1.RecordMarkerCommandAttributes.details:type_name -> temporal.api.command.v1.RecordMarkerCommandAttributes.DetailsEntry + 22, // 19: temporal.api.command.v1.RecordMarkerCommandAttributes.header:type_name -> temporal.api.common.v1.Header + 26, // 20: temporal.api.command.v1.RecordMarkerCommandAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 30, // 21: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 21, // 22: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 23, // 23: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes.input:type_name -> temporal.api.common.v1.Payloads + 24, // 24: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes.workflow_run_timeout:type_name -> google.protobuf.Duration + 24, // 25: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes.workflow_task_timeout:type_name -> google.protobuf.Duration + 24, // 26: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes.backoff_start_interval:type_name -> google.protobuf.Duration + 25, // 27: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes.retry_policy:type_name -> temporal.api.common.v1.RetryPolicy + 31, // 28: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes.initiator:type_name -> temporal.api.enums.v1.ContinueAsNewInitiator + 26, // 29: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 23, // 30: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes.last_completion_result:type_name -> temporal.api.common.v1.Payloads + 22, // 31: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes.header:type_name -> temporal.api.common.v1.Header + 29, // 32: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes.memo:type_name -> temporal.api.common.v1.Memo + 28, // 33: temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 30, // 34: temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 21, // 35: temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 23, // 36: temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes.input:type_name -> temporal.api.common.v1.Payloads + 24, // 37: temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes.workflow_execution_timeout:type_name -> google.protobuf.Duration + 24, // 38: temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes.workflow_run_timeout:type_name -> google.protobuf.Duration + 24, // 39: temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes.workflow_task_timeout:type_name -> google.protobuf.Duration + 32, // 40: temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes.parent_close_policy:type_name -> temporal.api.enums.v1.ParentClosePolicy + 33, // 41: temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes.workflow_id_reuse_policy:type_name -> temporal.api.enums.v1.WorkflowIdReusePolicy + 25, // 42: temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes.retry_policy:type_name -> temporal.api.common.v1.RetryPolicy + 22, // 43: temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes.header:type_name -> temporal.api.common.v1.Header + 29, // 44: temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes.memo:type_name -> temporal.api.common.v1.Memo + 28, // 45: temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 34, // 46: temporal.api.command.v1.ScheduleNexusOperationCommandAttributes.input:type_name -> temporal.api.common.v1.Payload + 24, // 47: temporal.api.command.v1.ScheduleNexusOperationCommandAttributes.schedule_to_close_timeout:type_name -> google.protobuf.Duration + 19, // 48: temporal.api.command.v1.ScheduleNexusOperationCommandAttributes.nexus_header:type_name -> temporal.api.command.v1.ScheduleNexusOperationCommandAttributes.NexusHeaderEntry + 35, // 49: temporal.api.command.v1.Command.command_type:type_name -> temporal.api.enums.v1.CommandType + 36, // 50: temporal.api.command.v1.Command.user_metadata:type_name -> temporal.api.sdk.v1.UserMetadata + 0, // 51: temporal.api.command.v1.Command.schedule_activity_task_command_attributes:type_name -> temporal.api.command.v1.ScheduleActivityTaskCommandAttributes + 2, // 52: temporal.api.command.v1.Command.start_timer_command_attributes:type_name -> temporal.api.command.v1.StartTimerCommandAttributes + 3, // 53: temporal.api.command.v1.Command.complete_workflow_execution_command_attributes:type_name -> temporal.api.command.v1.CompleteWorkflowExecutionCommandAttributes + 4, // 54: temporal.api.command.v1.Command.fail_workflow_execution_command_attributes:type_name -> temporal.api.command.v1.FailWorkflowExecutionCommandAttributes + 1, // 55: temporal.api.command.v1.Command.request_cancel_activity_task_command_attributes:type_name -> temporal.api.command.v1.RequestCancelActivityTaskCommandAttributes + 5, // 56: temporal.api.command.v1.Command.cancel_timer_command_attributes:type_name -> temporal.api.command.v1.CancelTimerCommandAttributes + 6, // 57: temporal.api.command.v1.Command.cancel_workflow_execution_command_attributes:type_name -> temporal.api.command.v1.CancelWorkflowExecutionCommandAttributes + 7, // 58: temporal.api.command.v1.Command.request_cancel_external_workflow_execution_command_attributes:type_name -> temporal.api.command.v1.RequestCancelExternalWorkflowExecutionCommandAttributes + 11, // 59: temporal.api.command.v1.Command.record_marker_command_attributes:type_name -> temporal.api.command.v1.RecordMarkerCommandAttributes + 12, // 60: temporal.api.command.v1.Command.continue_as_new_workflow_execution_command_attributes:type_name -> temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes + 13, // 61: temporal.api.command.v1.Command.start_child_workflow_execution_command_attributes:type_name -> temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes + 8, // 62: temporal.api.command.v1.Command.signal_external_workflow_execution_command_attributes:type_name -> temporal.api.command.v1.SignalExternalWorkflowExecutionCommandAttributes + 9, // 63: temporal.api.command.v1.Command.upsert_workflow_search_attributes_command_attributes:type_name -> temporal.api.command.v1.UpsertWorkflowSearchAttributesCommandAttributes + 14, // 64: temporal.api.command.v1.Command.protocol_message_command_attributes:type_name -> temporal.api.command.v1.ProtocolMessageCommandAttributes + 10, // 65: temporal.api.command.v1.Command.modify_workflow_properties_command_attributes:type_name -> temporal.api.command.v1.ModifyWorkflowPropertiesCommandAttributes + 15, // 66: temporal.api.command.v1.Command.schedule_nexus_operation_command_attributes:type_name -> temporal.api.command.v1.ScheduleNexusOperationCommandAttributes + 16, // 67: temporal.api.command.v1.Command.request_cancel_nexus_operation_command_attributes:type_name -> temporal.api.command.v1.RequestCancelNexusOperationCommandAttributes + 23, // 68: temporal.api.command.v1.RecordMarkerCommandAttributes.DetailsEntry.value:type_name -> temporal.api.common.v1.Payloads + 69, // [69:69] is the sub-list for method output_type + 69, // [69:69] is the sub-list for method input_type + 69, // [69:69] is the sub-list for extension type_name + 69, // [69:69] is the sub-list for extension extendee + 0, // [0:69] is the sub-list for field type_name +} + +func init() { file_temporal_api_command_v1_message_proto_init() } +func file_temporal_api_command_v1_message_proto_init() { + if File_temporal_api_command_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_command_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*ScheduleActivityTaskCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*RequestCancelActivityTaskCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*StartTimerCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*CompleteWorkflowExecutionCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*FailWorkflowExecutionCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*CancelTimerCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*CancelWorkflowExecutionCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*RequestCancelExternalWorkflowExecutionCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*SignalExternalWorkflowExecutionCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*UpsertWorkflowSearchAttributesCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*ModifyWorkflowPropertiesCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[11].Exporter = func(v any, i int) any { + switch v := v.(*RecordMarkerCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[12].Exporter = func(v any, i int) any { + switch v := v.(*ContinueAsNewWorkflowExecutionCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[13].Exporter = func(v any, i int) any { + switch v := v.(*StartChildWorkflowExecutionCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[14].Exporter = func(v any, i int) any { + switch v := v.(*ProtocolMessageCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[15].Exporter = func(v any, i int) any { + switch v := v.(*ScheduleNexusOperationCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[16].Exporter = func(v any, i int) any { + switch v := v.(*RequestCancelNexusOperationCommandAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_command_v1_message_proto_msgTypes[17].Exporter = func(v any, i int) any { + switch v := v.(*Command); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_temporal_api_command_v1_message_proto_msgTypes[17].OneofWrappers = []any{ + (*Command_ScheduleActivityTaskCommandAttributes)(nil), + (*Command_StartTimerCommandAttributes)(nil), + (*Command_CompleteWorkflowExecutionCommandAttributes)(nil), + (*Command_FailWorkflowExecutionCommandAttributes)(nil), + (*Command_RequestCancelActivityTaskCommandAttributes)(nil), + (*Command_CancelTimerCommandAttributes)(nil), + (*Command_CancelWorkflowExecutionCommandAttributes)(nil), + (*Command_RequestCancelExternalWorkflowExecutionCommandAttributes)(nil), + (*Command_RecordMarkerCommandAttributes)(nil), + (*Command_ContinueAsNewWorkflowExecutionCommandAttributes)(nil), + (*Command_StartChildWorkflowExecutionCommandAttributes)(nil), + (*Command_SignalExternalWorkflowExecutionCommandAttributes)(nil), + (*Command_UpsertWorkflowSearchAttributesCommandAttributes)(nil), + (*Command_ProtocolMessageCommandAttributes)(nil), + (*Command_ModifyWorkflowPropertiesCommandAttributes)(nil), + (*Command_ScheduleNexusOperationCommandAttributes)(nil), + (*Command_RequestCancelNexusOperationCommandAttributes)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_command_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 20, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_command_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_command_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_command_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_command_v1_message_proto = out.File + file_temporal_api_command_v1_message_proto_rawDesc = nil + file_temporal_api_command_v1_message_proto_goTypes = nil + file_temporal_api_command_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/common/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/common/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..03d9f26b22d --- /dev/null +++ b/vendor/go.temporal.io/api/common/v1/message.go-helpers.pb.go @@ -0,0 +1,620 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package common + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type DataBlob to the protobuf v3 wire format +func (val *DataBlob) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DataBlob from the protobuf v3 wire format +func (val *DataBlob) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DataBlob) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DataBlob values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DataBlob) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DataBlob + switch t := that.(type) { + case *DataBlob: + that1 = t + case DataBlob: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Payloads to the protobuf v3 wire format +func (val *Payloads) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Payloads from the protobuf v3 wire format +func (val *Payloads) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Payloads) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Payloads values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Payloads) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Payloads + switch t := that.(type) { + case *Payloads: + that1 = t + case Payloads: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Payload to the protobuf v3 wire format +func (val *Payload) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Payload from the protobuf v3 wire format +func (val *Payload) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Payload) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Payload values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Payload) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Payload + switch t := that.(type) { + case *Payload: + that1 = t + case Payload: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SearchAttributes to the protobuf v3 wire format +func (val *SearchAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SearchAttributes from the protobuf v3 wire format +func (val *SearchAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SearchAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SearchAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SearchAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SearchAttributes + switch t := that.(type) { + case *SearchAttributes: + that1 = t + case SearchAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Memo to the protobuf v3 wire format +func (val *Memo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Memo from the protobuf v3 wire format +func (val *Memo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Memo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Memo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Memo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Memo + switch t := that.(type) { + case *Memo: + that1 = t + case Memo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Header to the protobuf v3 wire format +func (val *Header) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Header from the protobuf v3 wire format +func (val *Header) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Header) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Header values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Header) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Header + switch t := that.(type) { + case *Header: + that1 = t + case Header: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecution to the protobuf v3 wire format +func (val *WorkflowExecution) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecution from the protobuf v3 wire format +func (val *WorkflowExecution) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecution) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecution values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecution) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecution + switch t := that.(type) { + case *WorkflowExecution: + that1 = t + case WorkflowExecution: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowType to the protobuf v3 wire format +func (val *WorkflowType) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowType from the protobuf v3 wire format +func (val *WorkflowType) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowType) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowType values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowType) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowType + switch t := that.(type) { + case *WorkflowType: + that1 = t + case WorkflowType: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ActivityType to the protobuf v3 wire format +func (val *ActivityType) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ActivityType from the protobuf v3 wire format +func (val *ActivityType) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ActivityType) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ActivityType values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ActivityType) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ActivityType + switch t := that.(type) { + case *ActivityType: + that1 = t + case ActivityType: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RetryPolicy to the protobuf v3 wire format +func (val *RetryPolicy) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RetryPolicy from the protobuf v3 wire format +func (val *RetryPolicy) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RetryPolicy) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RetryPolicy values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RetryPolicy) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RetryPolicy + switch t := that.(type) { + case *RetryPolicy: + that1 = t + case RetryPolicy: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type MeteringMetadata to the protobuf v3 wire format +func (val *MeteringMetadata) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type MeteringMetadata from the protobuf v3 wire format +func (val *MeteringMetadata) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *MeteringMetadata) Size() int { + return proto.Size(val) +} + +// Equal returns whether two MeteringMetadata values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *MeteringMetadata) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *MeteringMetadata + switch t := that.(type) { + case *MeteringMetadata: + that1 = t + case MeteringMetadata: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkerVersionStamp to the protobuf v3 wire format +func (val *WorkerVersionStamp) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkerVersionStamp from the protobuf v3 wire format +func (val *WorkerVersionStamp) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkerVersionStamp) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkerVersionStamp values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkerVersionStamp) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkerVersionStamp + switch t := that.(type) { + case *WorkerVersionStamp: + that1 = t + case WorkerVersionStamp: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkerVersionCapabilities to the protobuf v3 wire format +func (val *WorkerVersionCapabilities) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkerVersionCapabilities from the protobuf v3 wire format +func (val *WorkerVersionCapabilities) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkerVersionCapabilities) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkerVersionCapabilities values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkerVersionCapabilities) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkerVersionCapabilities + switch t := that.(type) { + case *WorkerVersionCapabilities: + that1 = t + case WorkerVersionCapabilities: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ResetOptions to the protobuf v3 wire format +func (val *ResetOptions) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ResetOptions from the protobuf v3 wire format +func (val *ResetOptions) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ResetOptions) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ResetOptions values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ResetOptions) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ResetOptions + switch t := that.(type) { + case *ResetOptions: + that1 = t + case ResetOptions: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Callback to the protobuf v3 wire format +func (val *Callback) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Callback from the protobuf v3 wire format +func (val *Callback) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Callback) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Callback values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Callback) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Callback + switch t := that.(type) { + case *Callback: + that1 = t + case Callback: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Link to the protobuf v3 wire format +func (val *Link) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Link from the protobuf v3 wire format +func (val *Link) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Link) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Link values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Link) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Link + switch t := that.(type) { + case *Link: + that1 = t + case Link: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/common/v1/message.pb.go b/vendor/go.temporal.io/api/common/v1/message.pb.go new file mode 100644 index 00000000000..b1dc45e6453 --- /dev/null +++ b/vendor/go.temporal.io/api/common/v1/message.pb.go @@ -0,0 +1,1909 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/common/v1/message.proto + +package common + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/enums/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + durationpb "google.golang.org/protobuf/types/known/durationpb" + emptypb "google.golang.org/protobuf/types/known/emptypb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type DataBlob struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + EncodingType v1.EncodingType `protobuf:"varint,1,opt,name=encoding_type,json=encodingType,proto3,enum=temporal.api.enums.v1.EncodingType" json:"encoding_type,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` +} + +func (x *DataBlob) Reset() { + *x = DataBlob{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DataBlob) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DataBlob) ProtoMessage() {} + +func (x *DataBlob) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DataBlob.ProtoReflect.Descriptor instead. +func (*DataBlob) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *DataBlob) GetEncodingType() v1.EncodingType { + if x != nil { + return x.EncodingType + } + return v1.EncodingType(0) +} + +func (x *DataBlob) GetData() []byte { + if x != nil { + return x.Data + } + return nil +} + +// See `Payload` +type Payloads struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Payloads []*Payload `protobuf:"bytes,1,rep,name=payloads,proto3" json:"payloads,omitempty"` +} + +func (x *Payloads) Reset() { + *x = Payloads{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Payloads) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Payloads) ProtoMessage() {} + +func (x *Payloads) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Payloads.ProtoReflect.Descriptor instead. +func (*Payloads) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *Payloads) GetPayloads() []*Payload { + if x != nil { + return x.Payloads + } + return nil +} + +// Represents some binary (byte array) data (ex: activity input parameters or workflow result) with +// metadata which describes this binary data (format, encoding, encryption, etc). Serialization +// of the data may be user-defined. +type Payload struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Metadata map[string][]byte `protobuf:"bytes,1,rep,name=metadata,proto3" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` +} + +func (x *Payload) Reset() { + *x = Payload{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Payload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Payload) ProtoMessage() {} + +func (x *Payload) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Payload.ProtoReflect.Descriptor instead. +func (*Payload) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *Payload) GetMetadata() map[string][]byte { + if x != nil { + return x.Metadata + } + return nil +} + +func (x *Payload) GetData() []byte { + if x != nil { + return x.Data + } + return nil +} + +// A user-defined set of *indexed* fields that are used/exposed when listing/searching workflows. +// The payload is not serialized in a user-defined way. +type SearchAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + IndexedFields map[string]*Payload `protobuf:"bytes,1,rep,name=indexed_fields,json=indexedFields,proto3" json:"indexed_fields,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *SearchAttributes) Reset() { + *x = SearchAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SearchAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SearchAttributes) ProtoMessage() {} + +func (x *SearchAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SearchAttributes.ProtoReflect.Descriptor instead. +func (*SearchAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *SearchAttributes) GetIndexedFields() map[string]*Payload { + if x != nil { + return x.IndexedFields + } + return nil +} + +// A user-defined set of *unindexed* fields that are exposed when listing/searching workflows +type Memo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Fields map[string]*Payload `protobuf:"bytes,1,rep,name=fields,proto3" json:"fields,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *Memo) Reset() { + *x = Memo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Memo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Memo) ProtoMessage() {} + +func (x *Memo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Memo.ProtoReflect.Descriptor instead. +func (*Memo) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *Memo) GetFields() map[string]*Payload { + if x != nil { + return x.Fields + } + return nil +} + +// Contains metadata that can be attached to a variety of requests, like starting a workflow, and +// can be propagated between, for example, workflows and activities. +type Header struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Fields map[string]*Payload `protobuf:"bytes,1,rep,name=fields,proto3" json:"fields,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *Header) Reset() { + *x = Header{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Header) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Header) ProtoMessage() {} + +func (x *Header) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Header.ProtoReflect.Descriptor instead. +func (*Header) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{5} +} + +func (x *Header) GetFields() map[string]*Payload { + if x != nil { + return x.Fields + } + return nil +} + +// Identifies a specific workflow within a namespace. Practically speaking, because run_id is a +// uuid, a workflow execution is globally unique. Note that many commands allow specifying an empty +// run id as a way of saying "target the latest run of the workflow". +type WorkflowExecution struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + WorkflowId string `protobuf:"bytes,1,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + RunId string `protobuf:"bytes,2,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` +} + +func (x *WorkflowExecution) Reset() { + *x = WorkflowExecution{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecution) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecution) ProtoMessage() {} + +func (x *WorkflowExecution) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecution.ProtoReflect.Descriptor instead. +func (*WorkflowExecution) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{6} +} + +func (x *WorkflowExecution) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *WorkflowExecution) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +// Represents the identifier used by a workflow author to define the workflow. Typically, the +// name of a function. This is sometimes referred to as the workflow's "name" +type WorkflowType struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` +} + +func (x *WorkflowType) Reset() { + *x = WorkflowType{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowType) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowType) ProtoMessage() {} + +func (x *WorkflowType) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowType.ProtoReflect.Descriptor instead. +func (*WorkflowType) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{7} +} + +func (x *WorkflowType) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +// Represents the identifier used by a activity author to define the activity. Typically, the +// name of a function. This is sometimes referred to as the activity's "name" +type ActivityType struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` +} + +func (x *ActivityType) Reset() { + *x = ActivityType{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActivityType) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActivityType) ProtoMessage() {} + +func (x *ActivityType) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActivityType.ProtoReflect.Descriptor instead. +func (*ActivityType) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{8} +} + +func (x *ActivityType) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +// How retries ought to be handled, usable by both workflows and activities +type RetryPolicy struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Interval of the first retry. If retryBackoffCoefficient is 1.0 then it is used for all retries. + InitialInterval *durationpb.Duration `protobuf:"bytes,1,opt,name=initial_interval,json=initialInterval,proto3" json:"initial_interval,omitempty"` + // Coefficient used to calculate the next retry interval. + // The next retry interval is previous interval multiplied by the coefficient. + // Must be 1 or larger. + BackoffCoefficient float64 `protobuf:"fixed64,2,opt,name=backoff_coefficient,json=backoffCoefficient,proto3" json:"backoff_coefficient,omitempty"` + // Maximum interval between retries. Exponential backoff leads to interval increase. + // This value is the cap of the increase. Default is 100x of the initial interval. + MaximumInterval *durationpb.Duration `protobuf:"bytes,3,opt,name=maximum_interval,json=maximumInterval,proto3" json:"maximum_interval,omitempty"` + // Maximum number of attempts. When exceeded the retries stop even if not expired yet. + // 1 disables retries. 0 means unlimited (up to the timeouts) + MaximumAttempts int32 `protobuf:"varint,4,opt,name=maximum_attempts,json=maximumAttempts,proto3" json:"maximum_attempts,omitempty"` + // Non-Retryable errors types. Will stop retrying if the error type matches this list. Note that + // this is not a substring match, the error *type* (not message) must match exactly. + NonRetryableErrorTypes []string `protobuf:"bytes,5,rep,name=non_retryable_error_types,json=nonRetryableErrorTypes,proto3" json:"non_retryable_error_types,omitempty"` +} + +func (x *RetryPolicy) Reset() { + *x = RetryPolicy{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RetryPolicy) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RetryPolicy) ProtoMessage() {} + +func (x *RetryPolicy) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RetryPolicy.ProtoReflect.Descriptor instead. +func (*RetryPolicy) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{9} +} + +func (x *RetryPolicy) GetInitialInterval() *durationpb.Duration { + if x != nil { + return x.InitialInterval + } + return nil +} + +func (x *RetryPolicy) GetBackoffCoefficient() float64 { + if x != nil { + return x.BackoffCoefficient + } + return 0 +} + +func (x *RetryPolicy) GetMaximumInterval() *durationpb.Duration { + if x != nil { + return x.MaximumInterval + } + return nil +} + +func (x *RetryPolicy) GetMaximumAttempts() int32 { + if x != nil { + return x.MaximumAttempts + } + return 0 +} + +func (x *RetryPolicy) GetNonRetryableErrorTypes() []string { + if x != nil { + return x.NonRetryableErrorTypes + } + return nil +} + +// Metadata relevant for metering purposes +type MeteringMetadata struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Count of local activities which have begun an execution attempt during this workflow task, + // and whose first attempt occurred in some previous task. This is used for metering + // purposes, and does not affect workflow state. + // + // (-- api-linter: core::0141::forbidden-types=disabled + // + // aip.dev/not-precedent: Negative values make no sense to represent. --) + NonfirstLocalActivityExecutionAttempts uint32 `protobuf:"varint,13,opt,name=nonfirst_local_activity_execution_attempts,json=nonfirstLocalActivityExecutionAttempts,proto3" json:"nonfirst_local_activity_execution_attempts,omitempty"` +} + +func (x *MeteringMetadata) Reset() { + *x = MeteringMetadata{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MeteringMetadata) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MeteringMetadata) ProtoMessage() {} + +func (x *MeteringMetadata) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MeteringMetadata.ProtoReflect.Descriptor instead. +func (*MeteringMetadata) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{10} +} + +func (x *MeteringMetadata) GetNonfirstLocalActivityExecutionAttempts() uint32 { + if x != nil { + return x.NonfirstLocalActivityExecutionAttempts + } + return 0 +} + +// Identifies the version(s) of a worker that processed a task +type WorkerVersionStamp struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // An opaque whole-worker identifier. Replaces the deprecated `binary_checksum` field when this + // message is included in requests which previously used that. + BuildId string `protobuf:"bytes,1,opt,name=build_id,json=buildId,proto3" json:"build_id,omitempty"` + // If set, the worker is opting in to worker versioning. Otherwise, this is used only as a + // marker for workflow reset points and the BuildIDs search attribute. + UseVersioning bool `protobuf:"varint,3,opt,name=use_versioning,json=useVersioning,proto3" json:"use_versioning,omitempty"` +} + +func (x *WorkerVersionStamp) Reset() { + *x = WorkerVersionStamp{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkerVersionStamp) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkerVersionStamp) ProtoMessage() {} + +func (x *WorkerVersionStamp) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkerVersionStamp.ProtoReflect.Descriptor instead. +func (*WorkerVersionStamp) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{11} +} + +func (x *WorkerVersionStamp) GetBuildId() string { + if x != nil { + return x.BuildId + } + return "" +} + +func (x *WorkerVersionStamp) GetUseVersioning() bool { + if x != nil { + return x.UseVersioning + } + return false +} + +// Identifies the version(s) that a worker is compatible with when polling or identifying itself, +// and whether or not this worker is opting into the build-id based versioning feature. This is +// used by matching to determine which workers ought to receive what tasks. +type WorkerVersionCapabilities struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // An opaque whole-worker identifier + BuildId string `protobuf:"bytes,1,opt,name=build_id,json=buildId,proto3" json:"build_id,omitempty"` + // If set, the worker is opting in to worker versioning, and wishes to only receive appropriate + // tasks. + UseVersioning bool `protobuf:"varint,2,opt,name=use_versioning,json=useVersioning,proto3" json:"use_versioning,omitempty"` +} + +func (x *WorkerVersionCapabilities) Reset() { + *x = WorkerVersionCapabilities{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkerVersionCapabilities) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkerVersionCapabilities) ProtoMessage() {} + +func (x *WorkerVersionCapabilities) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkerVersionCapabilities.ProtoReflect.Descriptor instead. +func (*WorkerVersionCapabilities) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{12} +} + +func (x *WorkerVersionCapabilities) GetBuildId() string { + if x != nil { + return x.BuildId + } + return "" +} + +func (x *WorkerVersionCapabilities) GetUseVersioning() bool { + if x != nil { + return x.UseVersioning + } + return false +} + +// Describes where and how to reset a workflow, used for batch reset currently +// and may be used for single-workflow reset later. +type ResetOptions struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Which workflow task to reset to. + // + // Types that are assignable to Target: + // + // *ResetOptions_FirstWorkflowTask + // *ResetOptions_LastWorkflowTask + // *ResetOptions_WorkflowTaskId + // *ResetOptions_BuildId + Target isResetOptions_Target `protobuf_oneof:"target"` + // Event types to be reapplied (deprecated) + // Default: RESET_REAPPLY_TYPE_SIGNAL + ResetReapplyType v1.ResetReapplyType `protobuf:"varint,10,opt,name=reset_reapply_type,json=resetReapplyType,proto3,enum=temporal.api.enums.v1.ResetReapplyType" json:"reset_reapply_type,omitempty"` + // If true, limit the reset to only within the current run. (Applies to build_id targets and + // possibly others in the future.) + CurrentRunOnly bool `protobuf:"varint,11,opt,name=current_run_only,json=currentRunOnly,proto3" json:"current_run_only,omitempty"` + // Event types not to be reapplied + ResetReapplyExcludeTypes []v1.ResetReapplyExcludeType `protobuf:"varint,12,rep,packed,name=reset_reapply_exclude_types,json=resetReapplyExcludeTypes,proto3,enum=temporal.api.enums.v1.ResetReapplyExcludeType" json:"reset_reapply_exclude_types,omitempty"` +} + +func (x *ResetOptions) Reset() { + *x = ResetOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResetOptions) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResetOptions) ProtoMessage() {} + +func (x *ResetOptions) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResetOptions.ProtoReflect.Descriptor instead. +func (*ResetOptions) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{13} +} + +func (m *ResetOptions) GetTarget() isResetOptions_Target { + if m != nil { + return m.Target + } + return nil +} + +func (x *ResetOptions) GetFirstWorkflowTask() *emptypb.Empty { + if x, ok := x.GetTarget().(*ResetOptions_FirstWorkflowTask); ok { + return x.FirstWorkflowTask + } + return nil +} + +func (x *ResetOptions) GetLastWorkflowTask() *emptypb.Empty { + if x, ok := x.GetTarget().(*ResetOptions_LastWorkflowTask); ok { + return x.LastWorkflowTask + } + return nil +} + +func (x *ResetOptions) GetWorkflowTaskId() int64 { + if x, ok := x.GetTarget().(*ResetOptions_WorkflowTaskId); ok { + return x.WorkflowTaskId + } + return 0 +} + +func (x *ResetOptions) GetBuildId() string { + if x, ok := x.GetTarget().(*ResetOptions_BuildId); ok { + return x.BuildId + } + return "" +} + +func (x *ResetOptions) GetResetReapplyType() v1.ResetReapplyType { + if x != nil { + return x.ResetReapplyType + } + return v1.ResetReapplyType(0) +} + +func (x *ResetOptions) GetCurrentRunOnly() bool { + if x != nil { + return x.CurrentRunOnly + } + return false +} + +func (x *ResetOptions) GetResetReapplyExcludeTypes() []v1.ResetReapplyExcludeType { + if x != nil { + return x.ResetReapplyExcludeTypes + } + return nil +} + +type isResetOptions_Target interface { + isResetOptions_Target() +} + +type ResetOptions_FirstWorkflowTask struct { + // Resets to the first workflow task completed or started event. + FirstWorkflowTask *emptypb.Empty `protobuf:"bytes,1,opt,name=first_workflow_task,json=firstWorkflowTask,proto3,oneof"` +} + +type ResetOptions_LastWorkflowTask struct { + // Resets to the last workflow task completed or started event. + LastWorkflowTask *emptypb.Empty `protobuf:"bytes,2,opt,name=last_workflow_task,json=lastWorkflowTask,proto3,oneof"` +} + +type ResetOptions_WorkflowTaskId struct { + // The id of a specific `WORKFLOW_TASK_COMPLETED`,`WORKFLOW_TASK_TIMED_OUT`, `WORKFLOW_TASK_FAILED`, or + // `WORKFLOW_TASK_STARTED` event to reset to. + // Note that this option doesn't make sense when used as part of a batch request. + WorkflowTaskId int64 `protobuf:"varint,3,opt,name=workflow_task_id,json=workflowTaskId,proto3,oneof"` +} + +type ResetOptions_BuildId struct { + // Resets to the first workflow task processed by this build id. + // If the workflow was not processed by the build id, or the workflow task can't be + // determined, no reset will be performed. + // Note that by default, this reset is allowed to be to a prior run in a chain of + // continue-as-new. + BuildId string `protobuf:"bytes,4,opt,name=build_id,json=buildId,proto3,oneof"` +} + +func (*ResetOptions_FirstWorkflowTask) isResetOptions_Target() {} + +func (*ResetOptions_LastWorkflowTask) isResetOptions_Target() {} + +func (*ResetOptions_WorkflowTaskId) isResetOptions_Target() {} + +func (*ResetOptions_BuildId) isResetOptions_Target() {} + +// Callback to attach to various events in the system, e.g. workflow run completion. +type Callback struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Variant: + // + // *Callback_Nexus_ + // *Callback_Internal_ + Variant isCallback_Variant `protobuf_oneof:"variant"` +} + +func (x *Callback) Reset() { + *x = Callback{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Callback) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Callback) ProtoMessage() {} + +func (x *Callback) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Callback.ProtoReflect.Descriptor instead. +func (*Callback) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{14} +} + +func (m *Callback) GetVariant() isCallback_Variant { + if m != nil { + return m.Variant + } + return nil +} + +func (x *Callback) GetNexus() *Callback_Nexus { + if x, ok := x.GetVariant().(*Callback_Nexus_); ok { + return x.Nexus + } + return nil +} + +func (x *Callback) GetInternal() *Callback_Internal { + if x, ok := x.GetVariant().(*Callback_Internal_); ok { + return x.Internal + } + return nil +} + +type isCallback_Variant interface { + isCallback_Variant() +} + +type Callback_Nexus_ struct { + Nexus *Callback_Nexus `protobuf:"bytes,2,opt,name=nexus,proto3,oneof"` +} + +type Callback_Internal_ struct { + Internal *Callback_Internal `protobuf:"bytes,3,opt,name=internal,proto3,oneof"` +} + +func (*Callback_Nexus_) isCallback_Variant() {} + +func (*Callback_Internal_) isCallback_Variant() {} + +// Link can be associated with history events. It might contain information about an external entity +// related to the history event. For example, workflow A makes a Nexus call that starts workflow B: +// in this case, a history event in workflow A could contain a Link to the workflow started event in +// workflow B, and vice-versa. +type Link struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Variant: + // + // *Link_WorkflowEvent_ + Variant isLink_Variant `protobuf_oneof:"variant"` +} + +func (x *Link) Reset() { + *x = Link{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Link) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Link) ProtoMessage() {} + +func (x *Link) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[15] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Link.ProtoReflect.Descriptor instead. +func (*Link) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{15} +} + +func (m *Link) GetVariant() isLink_Variant { + if m != nil { + return m.Variant + } + return nil +} + +func (x *Link) GetWorkflowEvent() *Link_WorkflowEvent { + if x, ok := x.GetVariant().(*Link_WorkflowEvent_); ok { + return x.WorkflowEvent + } + return nil +} + +type isLink_Variant interface { + isLink_Variant() +} + +type Link_WorkflowEvent_ struct { + WorkflowEvent *Link_WorkflowEvent `protobuf:"bytes,1,opt,name=workflow_event,json=workflowEvent,proto3,oneof"` +} + +func (*Link_WorkflowEvent_) isLink_Variant() {} + +type Callback_Nexus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Callback URL. + Url string `protobuf:"bytes,1,opt,name=url,proto3" json:"url,omitempty"` + // Header to attach to callback request. + Header map[string]string `protobuf:"bytes,2,rep,name=header,proto3" json:"header,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *Callback_Nexus) Reset() { + *x = Callback_Nexus{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Callback_Nexus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Callback_Nexus) ProtoMessage() {} + +func (x *Callback_Nexus) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[20] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Callback_Nexus.ProtoReflect.Descriptor instead. +func (*Callback_Nexus) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{14, 0} +} + +func (x *Callback_Nexus) GetUrl() string { + if x != nil { + return x.Url + } + return "" +} + +func (x *Callback_Nexus) GetHeader() map[string]string { + if x != nil { + return x.Header + } + return nil +} + +// Callbacks to be delivered internally within the system. +// This variant is not settable in the API and will be rejected by the service with an INVALID_ARGUMENT error. +// The only reason that this is exposed is because callbacks are replicated across clusters via the +// WorkflowExecutionStarted event, which is defined in the public API. +type Callback_Internal struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Opaque internal data. + Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` +} + +func (x *Callback_Internal) Reset() { + *x = Callback_Internal{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Callback_Internal) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Callback_Internal) ProtoMessage() {} + +func (x *Callback_Internal) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[21] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Callback_Internal.ProtoReflect.Descriptor instead. +func (*Callback_Internal) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{14, 1} +} + +func (x *Callback_Internal) GetData() []byte { + if x != nil { + return x.Data + } + return nil +} + +type Link_WorkflowEvent struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + WorkflowId string `protobuf:"bytes,2,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + RunId string `protobuf:"bytes,3,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` + // Additional information about the workflow event. + // Eg: the caller workflow can send the history event details that made the Nexus call. + // + // Types that are assignable to Reference: + // + // *Link_WorkflowEvent_EventRef + Reference isLink_WorkflowEvent_Reference `protobuf_oneof:"reference"` +} + +func (x *Link_WorkflowEvent) Reset() { + *x = Link_WorkflowEvent{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Link_WorkflowEvent) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Link_WorkflowEvent) ProtoMessage() {} + +func (x *Link_WorkflowEvent) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[23] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Link_WorkflowEvent.ProtoReflect.Descriptor instead. +func (*Link_WorkflowEvent) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{15, 0} +} + +func (x *Link_WorkflowEvent) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *Link_WorkflowEvent) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *Link_WorkflowEvent) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +func (m *Link_WorkflowEvent) GetReference() isLink_WorkflowEvent_Reference { + if m != nil { + return m.Reference + } + return nil +} + +func (x *Link_WorkflowEvent) GetEventRef() *Link_WorkflowEvent_EventReference { + if x, ok := x.GetReference().(*Link_WorkflowEvent_EventRef); ok { + return x.EventRef + } + return nil +} + +type isLink_WorkflowEvent_Reference interface { + isLink_WorkflowEvent_Reference() +} + +type Link_WorkflowEvent_EventRef struct { + EventRef *Link_WorkflowEvent_EventReference `protobuf:"bytes,100,opt,name=event_ref,json=eventRef,proto3,oneof"` +} + +func (*Link_WorkflowEvent_EventRef) isLink_WorkflowEvent_Reference() {} + +type Link_WorkflowEvent_EventReference struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + EventId int64 `protobuf:"varint,1,opt,name=event_id,json=eventId,proto3" json:"event_id,omitempty"` + EventType v1.EventType `protobuf:"varint,2,opt,name=event_type,json=eventType,proto3,enum=temporal.api.enums.v1.EventType" json:"event_type,omitempty"` +} + +func (x *Link_WorkflowEvent_EventReference) Reset() { + *x = Link_WorkflowEvent_EventReference{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Link_WorkflowEvent_EventReference) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Link_WorkflowEvent_EventReference) ProtoMessage() {} + +func (x *Link_WorkflowEvent_EventReference) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_common_v1_message_proto_msgTypes[24] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Link_WorkflowEvent_EventReference.ProtoReflect.Descriptor instead. +func (*Link_WorkflowEvent_EventReference) Descriptor() ([]byte, []int) { + return file_temporal_api_common_v1_message_proto_rawDescGZIP(), []int{15, 0, 0} +} + +func (x *Link_WorkflowEvent_EventReference) GetEventId() int64 { + if x != nil { + return x.EventId + } + return 0 +} + +func (x *Link_WorkflowEvent_EventReference) GetEventType() v1.EventType { + if x != nil { + return x.EventType + } + return v1.EventType(0) +} + +var File_temporal_api_common_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_common_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x16, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x1a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x22, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x26, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x21, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, + 0x31, 0x2f, 0x72, 0x65, 0x73, 0x65, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x70, 0x0a, 0x08, + 0x44, 0x61, 0x74, 0x61, 0x42, 0x6c, 0x6f, 0x62, 0x12, 0x4c, 0x0a, 0x0d, 0x65, 0x6e, 0x63, 0x6f, 0x64, + 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x23, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, + 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x54, 0x79, 0x70, 0x65, + 0x52, 0x0c, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x16, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, + 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x22, 0x4b, 0x0a, 0x08, 0x50, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x73, 0x12, 0x3f, 0x0a, 0x08, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x52, 0x08, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x22, 0xb5, 0x01, 0x0a, 0x07, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4d, 0x0a, 0x08, 0x6d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x4d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, 0x16, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x43, 0x0a, 0x0d, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x18, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xe5, 0x01, + 0x0a, 0x10, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x12, 0x66, 0x0a, 0x0e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x5f, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x2e, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x1a, 0x69, 0x0a, 0x12, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x46, 0x69, 0x65, + 0x6c, 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x39, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xb0, 0x01, 0x0a, 0x04, 0x4d, 0x65, 0x6d, 0x6f, + 0x12, 0x44, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x6d, 0x6f, 0x2e, 0x46, 0x69, 0x65, 0x6c, + 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x1a, 0x62, 0x0a, 0x0b, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x39, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0xb4, 0x01, 0x0a, 0x06, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x46, 0x0a, 0x06, 0x66, 0x69, + 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x62, + 0x0a, 0x0b, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x39, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x53, 0x0a, 0x11, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x19, 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x72, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x26, 0x0a, 0x0c, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x26, 0x0a, 0x0c, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x16, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xc4, 0x02, 0x0a, 0x0b, 0x52, 0x65, 0x74, + 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x48, 0x0a, 0x10, 0x69, 0x6e, 0x69, 0x74, 0x69, + 0x61, 0x6c, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, + 0x61, 0x6c, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x33, 0x0a, + 0x13, 0x62, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x5f, 0x63, 0x6f, 0x65, 0x66, 0x66, 0x69, 0x63, 0x69, + 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x12, 0x62, 0x61, 0x63, 0x6b, 0x6f, 0x66, + 0x66, 0x43, 0x6f, 0x65, 0x66, 0x66, 0x69, 0x63, 0x69, 0x65, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x48, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, + 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x0f, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, + 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x5f, + 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x6d, + 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x3d, 0x0a, 0x19, 0x6e, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x16, 0x6e, 0x6f, 0x6e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x61, 0x62, 0x6c, 0x65, 0x45, + 0x72, 0x72, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x72, 0x0a, 0x10, + 0x4d, 0x65, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, + 0x5e, 0x0a, 0x2a, 0x6e, 0x6f, 0x6e, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x63, 0x61, 0x6c, + 0x5f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, + 0x52, 0x26, 0x6e, 0x6f, 0x6e, 0x66, 0x69, 0x72, 0x73, 0x74, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x41, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x74, + 0x74, 0x65, 0x6d, 0x70, 0x74, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x5e, 0x0a, 0x12, 0x57, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x1d, + 0x0a, 0x08, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, 0x0a, 0x0e, 0x75, + 0x73, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0d, 0x75, 0x73, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, + 0x42, 0x02, 0x68, 0x00, 0x22, 0x65, 0x0a, 0x19, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x12, + 0x1d, 0x0a, 0x08, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, 0x0a, 0x0e, + 0x75, 0x73, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x0d, 0x75, 0x73, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, + 0x67, 0x42, 0x02, 0x68, 0x00, 0x22, 0xff, 0x03, 0x0a, 0x0c, 0x52, 0x65, 0x73, 0x65, 0x74, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x13, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x48, 0x00, 0x52, 0x11, 0x66, 0x69, 0x72, 0x73, 0x74, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, + 0x0a, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, + 0x61, 0x73, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x48, + 0x00, 0x52, 0x10, 0x6c, 0x61, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, + 0x73, 0x6b, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2e, 0x0a, 0x10, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, + 0x52, 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x1f, 0x0a, 0x08, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x59, 0x0a, 0x12, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x61, 0x70, 0x70, + 0x6c, 0x79, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x54, + 0x79, 0x70, 0x65, 0x52, 0x10, 0x72, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x61, 0x70, 0x70, 0x6c, 0x79, + 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x63, 0x75, 0x72, 0x72, 0x65, + 0x6e, 0x74, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6e, 0x4f, 0x6e, 0x6c, 0x79, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x71, 0x0a, 0x1b, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x61, 0x70, + 0x70, 0x6c, 0x79, 0x5f, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, + 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, + 0x65, 0x74, 0x52, 0x65, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, + 0x79, 0x70, 0x65, 0x52, 0x18, 0x72, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x61, 0x70, 0x70, 0x6c, 0x79, + 0x45, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, 0x79, 0x70, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x42, + 0x08, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x22, 0x83, 0x03, 0x0a, 0x08, 0x43, 0x61, 0x6c, + 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x12, 0x42, 0x0a, 0x05, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x62, + 0x61, 0x63, 0x6b, 0x2e, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x48, 0x00, 0x52, 0x05, 0x6e, 0x65, 0x78, 0x75, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x08, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x43, + 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x48, + 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x1a, 0xb0, + 0x01, 0x0a, 0x05, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x12, 0x14, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4e, 0x0a, 0x06, + 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x2e, 0x4e, 0x65, 0x78, 0x75, + 0x73, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x68, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x41, 0x0a, 0x0b, 0x48, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x22, 0x0a, 0x08, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x12, 0x16, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x04, 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x42, 0x09, 0x0a, 0x07, 0x76, 0x61, 0x72, 0x69, + 0x61, 0x6e, 0x74, 0x4a, 0x04, 0x08, 0x01, 0x10, 0x02, 0x22, 0xbf, 0x03, 0x0a, 0x04, 0x4c, 0x69, 0x6e, + 0x6b, 0x12, 0x57, 0x0a, 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x4c, 0x69, 0x6e, 0x6b, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x48, 0x00, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x42, 0x02, 0x68, 0x00, 0x1a, 0xd2, 0x02, 0x0a, 0x0d, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x19, 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x5f, + 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x72, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x65, 0x66, 0x18, 0x64, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x6e, 0x6b, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x48, 0x00, 0x52, 0x08, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x66, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x74, 0x0a, 0x0e, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x1d, 0x0a, 0x08, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x43, 0x0a, 0x0a, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x20, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x42, 0x0b, 0x0a, 0x09, 0x72, 0x65, 0x66, + 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x76, 0x61, 0x72, 0x69, 0x61, 0x6e, 0x74, + 0x42, 0x89, 0x01, 0x0a, 0x19, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x23, 0x67, 0x6f, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x3b, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0xaa, 0x02, + 0x18, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1b, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x43, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_common_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_common_v1_message_proto_rawDescData = file_temporal_api_common_v1_message_proto_rawDesc +) + +func file_temporal_api_common_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_common_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_common_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_common_v1_message_proto_rawDescData) + }) + return file_temporal_api_common_v1_message_proto_rawDescData +} + +var file_temporal_api_common_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 25) +var file_temporal_api_common_v1_message_proto_goTypes = []any{ + (*DataBlob)(nil), // 0: temporal.api.common.v1.DataBlob + (*Payloads)(nil), // 1: temporal.api.common.v1.Payloads + (*Payload)(nil), // 2: temporal.api.common.v1.Payload + (*SearchAttributes)(nil), // 3: temporal.api.common.v1.SearchAttributes + (*Memo)(nil), // 4: temporal.api.common.v1.Memo + (*Header)(nil), // 5: temporal.api.common.v1.Header + (*WorkflowExecution)(nil), // 6: temporal.api.common.v1.WorkflowExecution + (*WorkflowType)(nil), // 7: temporal.api.common.v1.WorkflowType + (*ActivityType)(nil), // 8: temporal.api.common.v1.ActivityType + (*RetryPolicy)(nil), // 9: temporal.api.common.v1.RetryPolicy + (*MeteringMetadata)(nil), // 10: temporal.api.common.v1.MeteringMetadata + (*WorkerVersionStamp)(nil), // 11: temporal.api.common.v1.WorkerVersionStamp + (*WorkerVersionCapabilities)(nil), // 12: temporal.api.common.v1.WorkerVersionCapabilities + (*ResetOptions)(nil), // 13: temporal.api.common.v1.ResetOptions + (*Callback)(nil), // 14: temporal.api.common.v1.Callback + (*Link)(nil), // 15: temporal.api.common.v1.Link + nil, // 16: temporal.api.common.v1.Payload.MetadataEntry + nil, // 17: temporal.api.common.v1.SearchAttributes.IndexedFieldsEntry + nil, // 18: temporal.api.common.v1.Memo.FieldsEntry + nil, // 19: temporal.api.common.v1.Header.FieldsEntry + (*Callback_Nexus)(nil), // 20: temporal.api.common.v1.Callback.Nexus + (*Callback_Internal)(nil), // 21: temporal.api.common.v1.Callback.Internal + nil, // 22: temporal.api.common.v1.Callback.Nexus.HeaderEntry + (*Link_WorkflowEvent)(nil), // 23: temporal.api.common.v1.Link.WorkflowEvent + (*Link_WorkflowEvent_EventReference)(nil), // 24: temporal.api.common.v1.Link.WorkflowEvent.EventReference + (v1.EncodingType)(0), // 25: temporal.api.enums.v1.EncodingType + (*durationpb.Duration)(nil), // 26: google.protobuf.Duration + (*emptypb.Empty)(nil), // 27: google.protobuf.Empty + (v1.ResetReapplyType)(0), // 28: temporal.api.enums.v1.ResetReapplyType + (v1.ResetReapplyExcludeType)(0), // 29: temporal.api.enums.v1.ResetReapplyExcludeType + (v1.EventType)(0), // 30: temporal.api.enums.v1.EventType +} +var file_temporal_api_common_v1_message_proto_depIdxs = []int32{ + 25, // 0: temporal.api.common.v1.DataBlob.encoding_type:type_name -> temporal.api.enums.v1.EncodingType + 2, // 1: temporal.api.common.v1.Payloads.payloads:type_name -> temporal.api.common.v1.Payload + 16, // 2: temporal.api.common.v1.Payload.metadata:type_name -> temporal.api.common.v1.Payload.MetadataEntry + 17, // 3: temporal.api.common.v1.SearchAttributes.indexed_fields:type_name -> temporal.api.common.v1.SearchAttributes.IndexedFieldsEntry + 18, // 4: temporal.api.common.v1.Memo.fields:type_name -> temporal.api.common.v1.Memo.FieldsEntry + 19, // 5: temporal.api.common.v1.Header.fields:type_name -> temporal.api.common.v1.Header.FieldsEntry + 26, // 6: temporal.api.common.v1.RetryPolicy.initial_interval:type_name -> google.protobuf.Duration + 26, // 7: temporal.api.common.v1.RetryPolicy.maximum_interval:type_name -> google.protobuf.Duration + 27, // 8: temporal.api.common.v1.ResetOptions.first_workflow_task:type_name -> google.protobuf.Empty + 27, // 9: temporal.api.common.v1.ResetOptions.last_workflow_task:type_name -> google.protobuf.Empty + 28, // 10: temporal.api.common.v1.ResetOptions.reset_reapply_type:type_name -> temporal.api.enums.v1.ResetReapplyType + 29, // 11: temporal.api.common.v1.ResetOptions.reset_reapply_exclude_types:type_name -> temporal.api.enums.v1.ResetReapplyExcludeType + 20, // 12: temporal.api.common.v1.Callback.nexus:type_name -> temporal.api.common.v1.Callback.Nexus + 21, // 13: temporal.api.common.v1.Callback.internal:type_name -> temporal.api.common.v1.Callback.Internal + 23, // 14: temporal.api.common.v1.Link.workflow_event:type_name -> temporal.api.common.v1.Link.WorkflowEvent + 2, // 15: temporal.api.common.v1.SearchAttributes.IndexedFieldsEntry.value:type_name -> temporal.api.common.v1.Payload + 2, // 16: temporal.api.common.v1.Memo.FieldsEntry.value:type_name -> temporal.api.common.v1.Payload + 2, // 17: temporal.api.common.v1.Header.FieldsEntry.value:type_name -> temporal.api.common.v1.Payload + 22, // 18: temporal.api.common.v1.Callback.Nexus.header:type_name -> temporal.api.common.v1.Callback.Nexus.HeaderEntry + 24, // 19: temporal.api.common.v1.Link.WorkflowEvent.event_ref:type_name -> temporal.api.common.v1.Link.WorkflowEvent.EventReference + 30, // 20: temporal.api.common.v1.Link.WorkflowEvent.EventReference.event_type:type_name -> temporal.api.enums.v1.EventType + 21, // [21:21] is the sub-list for method output_type + 21, // [21:21] is the sub-list for method input_type + 21, // [21:21] is the sub-list for extension type_name + 21, // [21:21] is the sub-list for extension extendee + 0, // [0:21] is the sub-list for field type_name +} + +func init() { file_temporal_api_common_v1_message_proto_init() } +func file_temporal_api_common_v1_message_proto_init() { + if File_temporal_api_common_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_common_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*DataBlob); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*Payloads); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*Payload); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*SearchAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*Memo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*Header); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecution); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowType); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*ActivityType); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*RetryPolicy); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*MeteringMetadata); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[11].Exporter = func(v any, i int) any { + switch v := v.(*WorkerVersionStamp); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[12].Exporter = func(v any, i int) any { + switch v := v.(*WorkerVersionCapabilities); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[13].Exporter = func(v any, i int) any { + switch v := v.(*ResetOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[14].Exporter = func(v any, i int) any { + switch v := v.(*Callback); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[15].Exporter = func(v any, i int) any { + switch v := v.(*Link); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[20].Exporter = func(v any, i int) any { + switch v := v.(*Callback_Nexus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[21].Exporter = func(v any, i int) any { + switch v := v.(*Callback_Internal); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[23].Exporter = func(v any, i int) any { + switch v := v.(*Link_WorkflowEvent); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_common_v1_message_proto_msgTypes[24].Exporter = func(v any, i int) any { + switch v := v.(*Link_WorkflowEvent_EventReference); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_temporal_api_common_v1_message_proto_msgTypes[13].OneofWrappers = []any{ + (*ResetOptions_FirstWorkflowTask)(nil), + (*ResetOptions_LastWorkflowTask)(nil), + (*ResetOptions_WorkflowTaskId)(nil), + (*ResetOptions_BuildId)(nil), + } + file_temporal_api_common_v1_message_proto_msgTypes[14].OneofWrappers = []any{ + (*Callback_Nexus_)(nil), + (*Callback_Internal_)(nil), + } + file_temporal_api_common_v1_message_proto_msgTypes[15].OneofWrappers = []any{ + (*Link_WorkflowEvent_)(nil), + } + file_temporal_api_common_v1_message_proto_msgTypes[23].OneofWrappers = []any{ + (*Link_WorkflowEvent_EventRef)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_common_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 25, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_common_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_common_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_common_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_common_v1_message_proto = out.File + file_temporal_api_common_v1_message_proto_rawDesc = nil + file_temporal_api_common_v1_message_proto_goTypes = nil + file_temporal_api_common_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/common/v1/payload_json.go b/vendor/go.temporal.io/api/common/v1/payload_json.go new file mode 100644 index 00000000000..7c39c81ed3f --- /dev/null +++ b/vendor/go.temporal.io/api/common/v1/payload_json.go @@ -0,0 +1,627 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package common + +import ( + //"bytes" + + "encoding" + "encoding/base64" + gojson "encoding/json" + "fmt" + "reflect" + "sort" + "strconv" + "strings" + + "go.temporal.io/api/internal/protojson" + "go.temporal.io/api/internal/protojson/json" + // "google.golang.org/protobuf/encoding/protojson" +) + +const ( + payloadMetadataKey = "metadata" + payloadDataKey = "data" + shorthandMessageTypeKey = "_protoMessageType" + binaryNullBase64 = "YmluYXJ5L251bGw=" +) + +var _ protojson.ProtoJSONMaybeMarshaler = (*Payload)(nil) +var _ protojson.ProtoJSONMaybeMarshaler = (*Payloads)(nil) +var _ protojson.ProtoJSONMaybeUnmarshaler = (*Payload)(nil) +var _ protojson.ProtoJSONMaybeUnmarshaler = (*Payloads)(nil) + +// !!! This file is copied from internal/temporalcommonv1 to common/v1. +// !!! DO NOT EDIT at common/v1/payload_json.go. +func marshalSingular(enc *json.Encoder, value interface{}) error { + switch vv := value.(type) { + case string: + return enc.WriteString(vv) + case bool: + enc.WriteBool(vv) + case int: + enc.WriteInt(int64(vv)) + case int64: + enc.WriteInt(vv) + case uint: + enc.WriteUint(uint64(vv)) + case uint64: + enc.WriteUint(vv) + case float32: + enc.WriteFloat(float64(vv), 32) + case float64: + enc.WriteFloat(vv, 64) + default: + return fmt.Errorf("cannot marshal type %[1]T value %[1]v", vv) + } + return nil +} + +func marshalStruct(enc *json.Encoder, vv reflect.Value) error { + enc.StartObject() + defer enc.EndObject() + ty := vv.Type() + +Loop: + for i, n := 0, vv.NumField(); i < n; i++ { + f := vv.Field(i) + name := f.String() + // lowercase. private field + if name[0] >= 'a' && name[0] <= 'z' { + continue + } + + // Handle encoding/json struct tags + tag, present := ty.Field(i).Tag.Lookup("json") + if present { + opts := strings.Split(tag, ",") + for j := range opts { + if opts[j] == "omitempty" && vv.IsZero() { + continue Loop + } else if opts[j] == "-" { + continue Loop + } + // name overridden + name = opts[j] + } + } + if err := enc.WriteName(name); err != nil { + return fmt.Errorf("unable to write name %s: %w", name, err) + } + if err := marshalSingular(enc, f.Interface()); err != nil { + return fmt.Errorf("unable to marshal value for name %s: %w", name, err) + } + } + return nil +} + +type keyVal struct { + k string + v reflect.Value +} + +// Map keys must be either strings or integers. We don't use encoding.TextMarshaler so we don't care +func marshalMap(enc *json.Encoder, vv reflect.Value) error { + enc.StartObject() + defer enc.EndObject() + + sv := make([]keyVal, vv.Len()) + iter := vv.MapRange() + for i := 0; iter.Next(); i++ { + k := iter.Key() + sv[i].v = iter.Value() + + if k.Kind() == reflect.String { + sv[i].k = k.String() + } else if tm, ok := k.Interface().(encoding.TextMarshaler); ok { + if k.Kind() == reflect.Pointer && k.IsNil() { + return nil + } + buf, err := tm.MarshalText() + sv[i].k = string(buf) + if err != nil { + return err + } + } else { + switch k.Kind() { + case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64: + sv[i].k = strconv.FormatInt(k.Int(), 10) + return nil + case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uintptr: + sv[i].k = strconv.FormatUint(k.Uint(), 10) + return nil + default: + return fmt.Errorf("map key type %T not supported", k) + } + } + } + // Sort keys just like encoding/json + sort.Slice(sv, func(i, j int) bool { return sv[i].k < sv[j].k }) + + for i := 0; i < len(sv); i++ { + if err := enc.WriteName(sv[i].k); err != nil { + return fmt.Errorf("unable to write name %s: %w", sv[i].k, err) + } + if err := marshalValue(enc, sv[i].v); err != nil { + return fmt.Errorf("unable to marshal value for name %s: %w", sv[i].k, err) + } + } + return nil +} + +func marshalValue(enc *json.Encoder, vv reflect.Value) error { + switch vv.Kind() { + case reflect.Slice, reflect.Array: + if vv.IsNil() || vv.Len() == 0 { + enc.WriteNull() + return nil + } + enc.StartArray() + defer enc.EndArray() + for i := 0; i < vv.Len(); i++ { + if err := marshalValue(enc, vv.Index(i)); err != nil { + return err + } + } + case reflect.Interface, reflect.Pointer: + if vv.IsNil() { + enc.WriteNull() + } else { + marshalValue(enc, vv.Elem()) + } + case reflect.Struct: + marshalStruct(enc, vv) + case reflect.Map: + if vv.IsNil() || vv.Len() == 0 { + enc.StartObject() + enc.EndObject() + return nil + } + marshalMap(enc, vv) + case reflect.Bool, + reflect.String, + reflect.Int, + reflect.Int8, + reflect.Int16, + reflect.Int32, + reflect.Int64, + reflect.Uint, + reflect.Uint8, + reflect.Uint16, + reflect.Uint32, + reflect.Uint64, + reflect.Uintptr, + reflect.Float32, + reflect.Float64, + reflect.Complex64, + reflect.Complex128: + marshalSingular(enc, vv.Interface()) + default: + return fmt.Errorf("cannot marshal %[1]T value %[1]v", vv.Interface()) + } + + return nil +} + +func marshal(enc *json.Encoder, value interface{}) error { + if value == nil { + // nil data means we send the binary/null encoding + enc.StartObject() + defer enc.EndObject() + if err := enc.WriteName("metadata"); err != nil { + return err + } + + enc.StartObject() + defer enc.EndObject() + if err := enc.WriteName("encoding"); err != nil { + return err + } + // base64(binary/null) + return enc.WriteString(binaryNullBase64) + } + return marshalValue(enc, reflect.ValueOf(value)) +} + +// Key on the marshaler metadata specifying whether shorthand is enabled. +// +// WARNING: This is internal API and should not be called externally. +const EnablePayloadShorthandMetadataKey = "__temporal_enable_payload_shorthand" + +// MaybeMarshalProtoJSON implements +// [go.temporal.io/api/internal/temporaljsonpb.ProtoJSONMaybeMarshaler.MaybeMarshalProtoJSON]. +// +// WARNING: This is internal API and should not be called externally. +func (p *Payloads) MaybeMarshalProtoJSON(meta map[string]interface{}, enc *json.Encoder) (handled bool, err error) { + // If this is nil, ignore + if p == nil { + return false, nil + } + + // Skip unless explicitly enabled + if _, enabled := meta[EnablePayloadShorthandMetadataKey].(bool); !enabled { + return false, nil + } + + // We only support marshalling to shorthand if all payloads are handled or + // there are no payloads, so check if all can be handled first. + vals := make([]any, len(p.Payloads)) + for i, payload := range p.Payloads { + handled, vals[i], err = payload.toJSONShorthand() + if !handled || err != nil { + return handled, err + } + } + + enc.StartArray() + defer enc.EndArray() + + for _, val := range vals { + if err = marshal(enc, val); err != nil { + return true, err + } + } + return true, err +} + +// MaybeUnmarshalProtoJSON implements +// [go.temporal.io/api/internal/temporaljsonpb.ProtoJSONMaybeUnmarshaler.MaybeUnmarshalProtoJSON]. +// +// WARNING: This is internal API and should not be called externally. +func (p *Payloads) MaybeUnmarshalProtoJSON(meta map[string]interface{}, dec *json.Decoder) (handled bool, err error) { + // If this is nil, ignore (should never be) + if p == nil { + return false, nil + } + // Skip unless explicitly enabled + if _, enabled := meta[EnablePayloadShorthandMetadataKey].(bool); !enabled { + return false, nil + } + tok, err := dec.Peek() + if err != nil { + return true, err + } + + if tok.Kind() == json.Null { + // Null is accepted as empty list + _, _ = dec.Read() + return true, nil + } else if tok.Kind() != json.ArrayOpen { + // If this isn't an array, then it's not shorthand + return false, nil + } + _, _ = dec.Read() + for { + tok, err := dec.Peek() + if err != nil { + return true, err + } + if tok.Kind() == json.ArrayClose { + _, _ = dec.Read() + break + } + var pl Payload + if err := pl.fromJSONMaybeShorthand(dec); err != nil { + return true, fmt.Errorf("unable to unmarshal payload: %w", err) + } + p.Payloads = append(p.Payloads, &pl) + } + + return true, nil +} + +// MaybeMarshalProtoJSON implements +// [go.temporal.io/api/internal/temporaljsonpb.ProtoJSONMaybeMarshaler.MaybeMarshalProtoJSON]. +// +// WARNING: This is internal API and should not be called externally. +func (p *Payload) MaybeMarshalProtoJSON(meta map[string]interface{}, enc *json.Encoder) (handled bool, err error) { + // If this is nil, ignore + if p == nil { + return false, nil + } + // Skip unless explicitly enabled + if _, enabled := meta[EnablePayloadShorthandMetadataKey].(bool); !enabled { + return false, nil + } + // If any are not handled or there is an error, return + handled, val, err := p.toJSONShorthand() + if !handled || err != nil { + return handled, err + } + return true, marshal(enc, val) +} + +// MaybeUnmarshalProtoJSON implements +// [go.temporal.io/api/internal/temporaljsonpb.ProtoJSONMaybeUnmarshaler.MaybeUnmarshalProtoJSON]. +// +// WARNING: This is internal API and should not be called externally. +func (p *Payload) MaybeUnmarshalProtoJSON(meta map[string]interface{}, dec *json.Decoder) (handled bool, err error) { + // If this is nil, ignore (should never be) + if p == nil { + return false, nil + } + // Skip unless explicitly enabled + if _, enabled := meta[EnablePayloadShorthandMetadataKey].(bool); !enabled { + return false, nil + } + // Always considered handled, unmarshaler ignored (unknown fields always + // disallowed for non-shorthand payloads at this time) + p.fromJSONMaybeShorthand(dec) + return true, nil +} + +func (p *Payload) toJSONShorthand() (handled bool, value interface{}, err error) { + // Only support binary null, plain JSON and proto JSON + switch string(p.Metadata["encoding"]) { + case "binary/null": + // Leave value as nil + handled = true + case "json/plain": + // Must only have this single metadata + if len(p.Metadata) != 1 { + return false, nil, nil + } + // We unmarshal because we may have to indent. We let this error fail the + // marshaller. + handled = true + err = gojson.Unmarshal(p.Data, &value) + case "json/protobuf": + // Must have the message type and no other metadata + msgType := string(p.Metadata["messageType"]) + if msgType == "" || len(p.Metadata) != 2 { + return false, nil, nil + } + // Since this is a proto object, this must unmarshal to a object. We let + // this error fail the marshaller. + var valueMap map[string]interface{} + handled = true + err = gojson.Unmarshal(p.Data, &valueMap) + // Put the message type on the object + if valueMap != nil { + valueMap[shorthandMessageTypeKey] = msgType + } + value = valueMap + default: + return false, nil, fmt.Errorf("unsupported encoding %s", string(p.Metadata["encoding"])) + } + return +} + +func unmarshalArray(dec *json.Decoder) (interface{}, error) { + var arr []interface{} + for { + tok, err := dec.Read() + if err != nil { + return nil, err + } + if tok.Kind() == json.ArrayClose { + return arr, nil + } + obj, err := unmarshalValue(dec, tok) + if err != nil { + return nil, err + } + arr = append(arr, obj) + } + +} + +func unmarshalValue(dec *json.Decoder, tok json.Token) (interface{}, error) { + switch tok.Kind() { + case json.Null: + return nil, nil + case json.Bool: + return tok.Bool(), nil + case json.Number: + i64, ok := tok.Int(64) + if ok { + return i64, nil + } + f64, ok := tok.Float(64) + if ok { + return f64, nil + } + return nil, fmt.Errorf("unable to parse number from %s", tok.Kind()) + case json.String: + return tok.ParsedString(), nil + case json.ObjectOpen: + out := map[string]interface{}{} + if err := unmarshalMap(dec, out); err != nil { + return nil, err + } + return out, nil + case json.ArrayOpen: + return unmarshalArray(dec) + default: + return nil, fmt.Errorf("unexpected %s token %v", tok.Kind(), tok) + } +} + +// Payloads are a map of string to things. All keys are strings however, so we can take shortcuts here. +func unmarshalMap(dec *json.Decoder, out map[string]interface{}) error { + for { + tok, err := dec.Read() + if err != nil { + return err + } + switch tok.Kind() { + default: + return fmt.Errorf("unexpected %s token", tok.Kind()) + case json.ObjectClose: + return nil + case json.Name: + key := tok.Name() + tok, err = dec.Read() + if err != nil { + return fmt.Errorf("unexpected error unmarshalling value for map key %q: %w", key, err) + } + val, err := unmarshalValue(dec, tok) + if err != nil { + return fmt.Errorf("unable to unmarshal value for map key %q: %w", key, err) + } + out[key] = val + } + } +} + +// Protojson marshals bytes as base64-encoded strings +func unmarshalBytes(s string) ([]byte, bool) { + enc := base64.StdEncoding + if strings.ContainsAny(s, "-_") { + enc = base64.URLEncoding + } + if len(s)%4 != 0 { + enc = enc.WithPadding(base64.NoPadding) + } + b, err := enc.DecodeString(s) + if err != nil { + return nil, false + } + return b, true +} + +// Attempt to unmarshal a standard payload from this map. Returns true if successful +func (p *Payload) unmarshalPayload(valueMap map[string]interface{}) bool { + md, mdOk := valueMap[payloadMetadataKey] + if !mdOk { + return false + } + + mdm, ok := md.(map[string]interface{}) + if !ok { + return false + } + + // Payloads must have an encoding + enc, ok := mdm["encoding"] + if !ok { + return false + } + + d, dataOk := valueMap[payloadDataKey] + // It's ok to have no data key if the encoding is binary/null + if mdOk && !dataOk && enc == binaryNullBase64 { + p.Metadata = map[string][]byte{ + "encoding": []byte("binary/null"), + } + return true + } else if !mdOk && !dataOk { + return false + } else if len(valueMap) > 2 { + // If we change the schema of the Payload type we'll need to update this + } + + // By this point payloads must have both data and metadata keys and no others + if !(dataOk && mdOk && len(valueMap) == 2) { + return false + } + + // We're probably a payload by this point + ds, ok := d.(string) + if !ok { + return false + } + + dataBytes, ok := unmarshalBytes(ds) + if !ok { + return false + } + mdbm := make(map[string][]byte, len(mdm)) + for k, v := range mdm { + vs, ok := v.(string) + // metadata keys will be encoded as base64 strings so we can reject everything else + if !ok { + return false + } + vb, ok := unmarshalBytes(vs) + if !ok { + return false + } + mdbm[k] = vb + } + + p.Metadata = mdbm + p.Data = dataBytes + return true +} + +func (p *Payload) fromJSONMaybeShorthand(dec *json.Decoder) error { + // We need to try to deserialize into the regular payload first. If it works + // and there is metadata _and_ data actually present (or null with a null + // metadata encoding), we assume it's a non-shorthand payload. If it fails + // (which it will if not an object or there is an unknown field or if + // 'metadata' is not string + base64 or if 'data' is not base64), we assume + // shorthand. We are ok disallowing unknown fields for payloads here even if + // the outer unmarshaler allows them. + tok, err := dec.Read() + if err != nil { + return err + } + val, err := unmarshalValue(dec, tok) + if err != nil { + return err + } + switch tv := val.(type) { + default: + // take it as-is + p.Metadata = map[string][]byte{"encoding": []byte("json/plain")} + p.Data, err = gojson.Marshal(val) + return err + case nil: + p.Data = nil + p.Metadata = map[string][]byte{"encoding": []byte("binary/null")} + return nil + case map[string]interface{}: + if handled := p.unmarshalPayload(tv); handled { + // Standard payload + return nil + } + + // Now that we know it is shorthand, it might be a proto JSON with a message + // type. If it does have the message type, we need to remove it and + // re-serialize it to data. So the quickest way to check whether it has the + // message type is to search for the key. + if maybeMsgType, found := tv[shorthandMessageTypeKey]; found { + msgType, ok := maybeMsgType.(string) + if !ok { + return fmt.Errorf("internal key %q should have type string, not %T", shorthandMessageTypeKey, maybeMsgType) + } + // Now we know it's a proto JSON, so remove the key and re-serialize + delete(tv, "_protoMessageType") + // This won't error. The resulting JSON payload data may not be exactly + // what user passed in sans message type (e.g. user may have indented or + // did not have same field order), but that is acceptable when going + // from shorthand to non-shorthand. + p.Data, _ = gojson.Marshal(tv) + p.Metadata = map[string][]byte{ + "encoding": []byte("json/protobuf"), + "messageType": []byte(msgType), + } + } else { + p.Metadata = map[string][]byte{"encoding": []byte("json/plain")} + p.Data, err = gojson.Marshal(val) + return err + } + return nil + } +} diff --git a/vendor/go.temporal.io/api/enums/v1/batch_operation.go-helpers.pb.go b/vendor/go.temporal.io/api/enums/v1/batch_operation.go-helpers.pb.go new file mode 100644 index 00000000000..e84e63f49ed --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/batch_operation.go-helpers.pb.go @@ -0,0 +1,70 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package enums + +import ( + "fmt" +) + +var ( + BatchOperationType_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Terminate": 1, + "Cancel": 2, + "Signal": 3, + "Delete": 4, + "Reset": 5, + } +) + +// BatchOperationTypeFromString parses a BatchOperationType value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to BatchOperationType +func BatchOperationTypeFromString(s string) (BatchOperationType, error) { + if v, ok := BatchOperationType_value[s]; ok { + return BatchOperationType(v), nil + } else if v, ok := BatchOperationType_shorthandValue[s]; ok { + return BatchOperationType(v), nil + } + return BatchOperationType(0), fmt.Errorf("%s is not a valid BatchOperationType", s) +} + +var ( + BatchOperationState_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Running": 1, + "Completed": 2, + "Failed": 3, + } +) + +// BatchOperationStateFromString parses a BatchOperationState value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to BatchOperationState +func BatchOperationStateFromString(s string) (BatchOperationState, error) { + if v, ok := BatchOperationState_value[s]; ok { + return BatchOperationState(v), nil + } else if v, ok := BatchOperationState_shorthandValue[s]; ok { + return BatchOperationState(v), nil + } + return BatchOperationState(0), fmt.Errorf("%s is not a valid BatchOperationState", s) +} diff --git a/vendor/go.temporal.io/api/enums/v1/batch_operation.pb.go b/vendor/go.temporal.io/api/enums/v1/batch_operation.pb.go new file mode 100644 index 00000000000..af1eb165db5 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/batch_operation.pb.go @@ -0,0 +1,277 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/enums/v1/batch_operation.proto + +package enums + +import ( + reflect "reflect" + "strconv" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type BatchOperationType int32 + +const ( + BATCH_OPERATION_TYPE_UNSPECIFIED BatchOperationType = 0 + BATCH_OPERATION_TYPE_TERMINATE BatchOperationType = 1 + BATCH_OPERATION_TYPE_CANCEL BatchOperationType = 2 + BATCH_OPERATION_TYPE_SIGNAL BatchOperationType = 3 + BATCH_OPERATION_TYPE_DELETE BatchOperationType = 4 + BATCH_OPERATION_TYPE_RESET BatchOperationType = 5 +) + +// Enum value maps for BatchOperationType. +var ( + BatchOperationType_name = map[int32]string{ + 0: "BATCH_OPERATION_TYPE_UNSPECIFIED", + 1: "BATCH_OPERATION_TYPE_TERMINATE", + 2: "BATCH_OPERATION_TYPE_CANCEL", + 3: "BATCH_OPERATION_TYPE_SIGNAL", + 4: "BATCH_OPERATION_TYPE_DELETE", + 5: "BATCH_OPERATION_TYPE_RESET", + } + BatchOperationType_value = map[string]int32{ + "BATCH_OPERATION_TYPE_UNSPECIFIED": 0, + "BATCH_OPERATION_TYPE_TERMINATE": 1, + "BATCH_OPERATION_TYPE_CANCEL": 2, + "BATCH_OPERATION_TYPE_SIGNAL": 3, + "BATCH_OPERATION_TYPE_DELETE": 4, + "BATCH_OPERATION_TYPE_RESET": 5, + } +) + +func (x BatchOperationType) Enum() *BatchOperationType { + p := new(BatchOperationType) + *p = x + return p +} + +func (x BatchOperationType) String() string { + switch x { + case BATCH_OPERATION_TYPE_UNSPECIFIED: + return "Unspecified" + case BATCH_OPERATION_TYPE_TERMINATE: + return "Terminate" + case BATCH_OPERATION_TYPE_CANCEL: + return "Cancel" + case BATCH_OPERATION_TYPE_SIGNAL: + return "Signal" + case BATCH_OPERATION_TYPE_DELETE: + return "Delete" + case BATCH_OPERATION_TYPE_RESET: + return "Reset" + default: + return strconv.Itoa(int(x)) + } + +} + +func (BatchOperationType) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_batch_operation_proto_enumTypes[0].Descriptor() +} + +func (BatchOperationType) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_batch_operation_proto_enumTypes[0] +} + +func (x BatchOperationType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use BatchOperationType.Descriptor instead. +func (BatchOperationType) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_batch_operation_proto_rawDescGZIP(), []int{0} +} + +type BatchOperationState int32 + +const ( + BATCH_OPERATION_STATE_UNSPECIFIED BatchOperationState = 0 + BATCH_OPERATION_STATE_RUNNING BatchOperationState = 1 + BATCH_OPERATION_STATE_COMPLETED BatchOperationState = 2 + BATCH_OPERATION_STATE_FAILED BatchOperationState = 3 +) + +// Enum value maps for BatchOperationState. +var ( + BatchOperationState_name = map[int32]string{ + 0: "BATCH_OPERATION_STATE_UNSPECIFIED", + 1: "BATCH_OPERATION_STATE_RUNNING", + 2: "BATCH_OPERATION_STATE_COMPLETED", + 3: "BATCH_OPERATION_STATE_FAILED", + } + BatchOperationState_value = map[string]int32{ + "BATCH_OPERATION_STATE_UNSPECIFIED": 0, + "BATCH_OPERATION_STATE_RUNNING": 1, + "BATCH_OPERATION_STATE_COMPLETED": 2, + "BATCH_OPERATION_STATE_FAILED": 3, + } +) + +func (x BatchOperationState) Enum() *BatchOperationState { + p := new(BatchOperationState) + *p = x + return p +} + +func (x BatchOperationState) String() string { + switch x { + case BATCH_OPERATION_STATE_UNSPECIFIED: + return "Unspecified" + case BATCH_OPERATION_STATE_RUNNING: + return "Running" + case BATCH_OPERATION_STATE_COMPLETED: + return "Completed" + case BATCH_OPERATION_STATE_FAILED: + return "Failed" + default: + return strconv.Itoa(int(x)) + } + +} + +func (BatchOperationState) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_batch_operation_proto_enumTypes[1].Descriptor() +} + +func (BatchOperationState) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_batch_operation_proto_enumTypes[1] +} + +func (x BatchOperationState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use BatchOperationState.Descriptor instead. +func (BatchOperationState) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_batch_operation_proto_rawDescGZIP(), []int{1} +} + +var File_temporal_api_enums_v1_batch_operation_proto protoreflect.FileDescriptor + +var file_temporal_api_enums_v1_batch_operation_proto_rawDesc = []byte{ + 0x0a, 0x2b, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x6f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, + 0x73, 0x2e, 0x76, 0x31, 0x2a, 0xe1, 0x01, 0x0a, 0x12, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x24, 0x0a, 0x20, 0x42, + 0x41, 0x54, 0x43, 0x48, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, + 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, + 0x00, 0x12, 0x22, 0x0a, 0x1e, 0x42, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, + 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x49, 0x4e, + 0x41, 0x54, 0x45, 0x10, 0x01, 0x12, 0x1f, 0x0a, 0x1b, 0x42, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x4f, + 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x41, + 0x4e, 0x43, 0x45, 0x4c, 0x10, 0x02, 0x12, 0x1f, 0x0a, 0x1b, 0x42, 0x41, 0x54, 0x43, 0x48, 0x5f, + 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, + 0x49, 0x47, 0x4e, 0x41, 0x4c, 0x10, 0x03, 0x12, 0x1f, 0x0a, 0x1b, 0x42, 0x41, 0x54, 0x43, 0x48, + 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, + 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x04, 0x12, 0x1e, 0x0a, 0x1a, 0x42, 0x41, 0x54, 0x43, + 0x48, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, + 0x5f, 0x52, 0x45, 0x53, 0x45, 0x54, 0x10, 0x05, 0x2a, 0xa6, 0x01, 0x0a, 0x13, 0x42, 0x61, 0x74, + 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x12, 0x25, 0x0a, 0x21, 0x42, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, + 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, + 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x21, 0x0a, 0x1d, 0x42, 0x41, 0x54, 0x43, 0x48, + 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, + 0x5f, 0x52, 0x55, 0x4e, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x23, 0x0a, 0x1f, 0x42, 0x41, + 0x54, 0x43, 0x48, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, + 0x41, 0x54, 0x45, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, + 0x20, 0x0a, 0x1c, 0x42, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, + 0x03, 0x42, 0x8b, 0x01, 0x0a, 0x18, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x42, 0x13, + 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, + 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x21, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, + 0x76, 0x31, 0x3b, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0xaa, 0x02, 0x17, 0x54, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x56, 0x31, 0xea, 0x02, 0x1a, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, + 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x3a, 0x3a, 0x56, 0x31, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_enums_v1_batch_operation_proto_rawDescOnce sync.Once + file_temporal_api_enums_v1_batch_operation_proto_rawDescData = file_temporal_api_enums_v1_batch_operation_proto_rawDesc +) + +func file_temporal_api_enums_v1_batch_operation_proto_rawDescGZIP() []byte { + file_temporal_api_enums_v1_batch_operation_proto_rawDescOnce.Do(func() { + file_temporal_api_enums_v1_batch_operation_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_enums_v1_batch_operation_proto_rawDescData) + }) + return file_temporal_api_enums_v1_batch_operation_proto_rawDescData +} + +var file_temporal_api_enums_v1_batch_operation_proto_enumTypes = make([]protoimpl.EnumInfo, 2) +var file_temporal_api_enums_v1_batch_operation_proto_goTypes = []any{ + (BatchOperationType)(0), // 0: temporal.api.enums.v1.BatchOperationType + (BatchOperationState)(0), // 1: temporal.api.enums.v1.BatchOperationState +} +var file_temporal_api_enums_v1_batch_operation_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_enums_v1_batch_operation_proto_init() } +func file_temporal_api_enums_v1_batch_operation_proto_init() { + if File_temporal_api_enums_v1_batch_operation_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_enums_v1_batch_operation_proto_rawDesc, + NumEnums: 2, + NumMessages: 0, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_enums_v1_batch_operation_proto_goTypes, + DependencyIndexes: file_temporal_api_enums_v1_batch_operation_proto_depIdxs, + EnumInfos: file_temporal_api_enums_v1_batch_operation_proto_enumTypes, + }.Build() + File_temporal_api_enums_v1_batch_operation_proto = out.File + file_temporal_api_enums_v1_batch_operation_proto_rawDesc = nil + file_temporal_api_enums_v1_batch_operation_proto_goTypes = nil + file_temporal_api_enums_v1_batch_operation_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/enums/v1/command_type.go-helpers.pb.go b/vendor/go.temporal.io/api/enums/v1/command_type.go-helpers.pb.go new file mode 100644 index 00000000000..c295c104d70 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/command_type.go-helpers.pb.go @@ -0,0 +1,62 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package enums + +import ( + "fmt" +) + +var ( + CommandType_shorthandValue = map[string]int32{ + "Unspecified": 0, + "ScheduleActivityTask": 1, + "RequestCancelActivityTask": 2, + "StartTimer": 3, + "CompleteWorkflowExecution": 4, + "FailWorkflowExecution": 5, + "CancelTimer": 6, + "CancelWorkflowExecution": 7, + "RequestCancelExternalWorkflowExecution": 8, + "RecordMarker": 9, + "ContinueAsNewWorkflowExecution": 10, + "StartChildWorkflowExecution": 11, + "SignalExternalWorkflowExecution": 12, + "UpsertWorkflowSearchAttributes": 13, + "ProtocolMessage": 14, + "ModifyWorkflowProperties": 16, + "ScheduleNexusOperation": 17, + "RequestCancelNexusOperation": 18, + } +) + +// CommandTypeFromString parses a CommandType value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to CommandType +func CommandTypeFromString(s string) (CommandType, error) { + if v, ok := CommandType_value[s]; ok { + return CommandType(v), nil + } else if v, ok := CommandType_shorthandValue[s]; ok { + return CommandType(v), nil + } + return CommandType(0), fmt.Errorf("%s is not a valid CommandType", s) +} diff --git a/vendor/go.temporal.io/api/enums/v1/command_type.pb.go b/vendor/go.temporal.io/api/enums/v1/command_type.pb.go new file mode 100644 index 00000000000..e04a2fee048 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/command_type.pb.go @@ -0,0 +1,299 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/enums/v1/command_type.proto + +package enums + +import ( + reflect "reflect" + "strconv" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// Whenever this list of command types is changed do change the function shouldBufferEvent in mutableStateBuilder.go to make sure to do the correct event ordering. +type CommandType int32 + +const ( + COMMAND_TYPE_UNSPECIFIED CommandType = 0 + COMMAND_TYPE_SCHEDULE_ACTIVITY_TASK CommandType = 1 + COMMAND_TYPE_REQUEST_CANCEL_ACTIVITY_TASK CommandType = 2 + COMMAND_TYPE_START_TIMER CommandType = 3 + COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION CommandType = 4 + COMMAND_TYPE_FAIL_WORKFLOW_EXECUTION CommandType = 5 + COMMAND_TYPE_CANCEL_TIMER CommandType = 6 + COMMAND_TYPE_CANCEL_WORKFLOW_EXECUTION CommandType = 7 + COMMAND_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION CommandType = 8 + COMMAND_TYPE_RECORD_MARKER CommandType = 9 + COMMAND_TYPE_CONTINUE_AS_NEW_WORKFLOW_EXECUTION CommandType = 10 + COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION CommandType = 11 + COMMAND_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION CommandType = 12 + COMMAND_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES CommandType = 13 + COMMAND_TYPE_PROTOCOL_MESSAGE CommandType = 14 + COMMAND_TYPE_MODIFY_WORKFLOW_PROPERTIES CommandType = 16 + COMMAND_TYPE_SCHEDULE_NEXUS_OPERATION CommandType = 17 + COMMAND_TYPE_REQUEST_CANCEL_NEXUS_OPERATION CommandType = 18 +) + +// Enum value maps for CommandType. +var ( + CommandType_name = map[int32]string{ + 0: "COMMAND_TYPE_UNSPECIFIED", + 1: "COMMAND_TYPE_SCHEDULE_ACTIVITY_TASK", + 2: "COMMAND_TYPE_REQUEST_CANCEL_ACTIVITY_TASK", + 3: "COMMAND_TYPE_START_TIMER", + 4: "COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION", + 5: "COMMAND_TYPE_FAIL_WORKFLOW_EXECUTION", + 6: "COMMAND_TYPE_CANCEL_TIMER", + 7: "COMMAND_TYPE_CANCEL_WORKFLOW_EXECUTION", + 8: "COMMAND_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION", + 9: "COMMAND_TYPE_RECORD_MARKER", + 10: "COMMAND_TYPE_CONTINUE_AS_NEW_WORKFLOW_EXECUTION", + 11: "COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION", + 12: "COMMAND_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION", + 13: "COMMAND_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES", + 14: "COMMAND_TYPE_PROTOCOL_MESSAGE", + 16: "COMMAND_TYPE_MODIFY_WORKFLOW_PROPERTIES", + 17: "COMMAND_TYPE_SCHEDULE_NEXUS_OPERATION", + 18: "COMMAND_TYPE_REQUEST_CANCEL_NEXUS_OPERATION", + } + CommandType_value = map[string]int32{ + "COMMAND_TYPE_UNSPECIFIED": 0, + "COMMAND_TYPE_SCHEDULE_ACTIVITY_TASK": 1, + "COMMAND_TYPE_REQUEST_CANCEL_ACTIVITY_TASK": 2, + "COMMAND_TYPE_START_TIMER": 3, + "COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION": 4, + "COMMAND_TYPE_FAIL_WORKFLOW_EXECUTION": 5, + "COMMAND_TYPE_CANCEL_TIMER": 6, + "COMMAND_TYPE_CANCEL_WORKFLOW_EXECUTION": 7, + "COMMAND_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION": 8, + "COMMAND_TYPE_RECORD_MARKER": 9, + "COMMAND_TYPE_CONTINUE_AS_NEW_WORKFLOW_EXECUTION": 10, + "COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION": 11, + "COMMAND_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION": 12, + "COMMAND_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES": 13, + "COMMAND_TYPE_PROTOCOL_MESSAGE": 14, + "COMMAND_TYPE_MODIFY_WORKFLOW_PROPERTIES": 16, + "COMMAND_TYPE_SCHEDULE_NEXUS_OPERATION": 17, + "COMMAND_TYPE_REQUEST_CANCEL_NEXUS_OPERATION": 18, + } +) + +func (x CommandType) Enum() *CommandType { + p := new(CommandType) + *p = x + return p +} + +func (x CommandType) String() string { + switch x { + case COMMAND_TYPE_UNSPECIFIED: + return "Unspecified" + case COMMAND_TYPE_SCHEDULE_ACTIVITY_TASK: + return "ScheduleActivityTask" + case COMMAND_TYPE_REQUEST_CANCEL_ACTIVITY_TASK: + return "RequestCancelActivityTask" + case COMMAND_TYPE_START_TIMER: + return "StartTimer" + case COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION: + return "CompleteWorkflowExecution" + case COMMAND_TYPE_FAIL_WORKFLOW_EXECUTION: + return "FailWorkflowExecution" + case COMMAND_TYPE_CANCEL_TIMER: + return "CancelTimer" + + // Deprecated: Use CommandType.Descriptor instead. + case COMMAND_TYPE_CANCEL_WORKFLOW_EXECUTION: + return "CancelWorkflowExecution" + case COMMAND_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION: + return "RequestCancelExternalWorkflowExecution" + case COMMAND_TYPE_RECORD_MARKER: + return "RecordMarker" + case COMMAND_TYPE_CONTINUE_AS_NEW_WORKFLOW_EXECUTION: + return "ContinueAsNewWorkflowExecution" + case COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION: + return "StartChildWorkflowExecution" + case COMMAND_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION: + return "SignalExternalWorkflowExecution" + case COMMAND_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES: + return "UpsertWorkflowSearchAttributes" + case COMMAND_TYPE_PROTOCOL_MESSAGE: + return "ProtocolMessage" + case COMMAND_TYPE_MODIFY_WORKFLOW_PROPERTIES: + return "ModifyWorkflowProperties" + case COMMAND_TYPE_SCHEDULE_NEXUS_OPERATION: + return "ScheduleNexusOperation" + case COMMAND_TYPE_REQUEST_CANCEL_NEXUS_OPERATION: + return "RequestCancelNexusOperation" + default: + return strconv.Itoa(int(x)) + } + +} + +func (CommandType) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_command_type_proto_enumTypes[0].Descriptor() +} + +func (CommandType) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_command_type_proto_enumTypes[0] +} + +func (x CommandType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +func (CommandType) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_command_type_proto_rawDescGZIP(), []int{0} +} + +var File_temporal_api_enums_v1_command_type_proto protoreflect.FileDescriptor + +var file_temporal_api_enums_v1_command_type_proto_rawDesc = []byte{ + 0x0a, 0x28, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, + 0x31, 0x2a, 0x9c, 0x06, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x79, 0x70, + 0x65, 0x12, 0x1c, 0x0a, 0x18, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x5f, 0x54, 0x59, 0x50, + 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, + 0x27, 0x0a, 0x23, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, + 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x56, 0x49, 0x54, + 0x59, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x10, 0x01, 0x12, 0x2d, 0x0a, 0x29, 0x43, 0x4f, 0x4d, 0x4d, + 0x41, 0x4e, 0x44, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, + 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, + 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x10, 0x02, 0x12, 0x1c, 0x0a, 0x18, 0x43, 0x4f, 0x4d, 0x4d, 0x41, + 0x4e, 0x44, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x54, 0x49, + 0x4d, 0x45, 0x52, 0x10, 0x03, 0x12, 0x2c, 0x0a, 0x28, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x5f, 0x57, + 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, + 0x4e, 0x10, 0x04, 0x12, 0x28, 0x0a, 0x24, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x5f, 0x54, + 0x59, 0x50, 0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, + 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x05, 0x12, 0x1d, 0x0a, + 0x19, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x41, + 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x52, 0x10, 0x06, 0x12, 0x2a, 0x0a, 0x26, + 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x41, 0x4e, + 0x43, 0x45, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, + 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x07, 0x12, 0x3b, 0x0a, 0x37, 0x43, 0x4f, 0x4d, 0x4d, + 0x41, 0x4e, 0x44, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, + 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, + 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, + 0x49, 0x4f, 0x4e, 0x10, 0x08, 0x12, 0x1e, 0x0a, 0x1a, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x52, 0x45, 0x43, 0x4f, 0x52, 0x44, 0x5f, 0x4d, 0x41, 0x52, + 0x4b, 0x45, 0x52, 0x10, 0x09, 0x12, 0x33, 0x0a, 0x2f, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x54, 0x49, 0x4e, 0x55, 0x45, 0x5f, 0x41, + 0x53, 0x5f, 0x4e, 0x45, 0x57, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, + 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x0a, 0x12, 0x2f, 0x0a, 0x2b, 0x43, 0x4f, + 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, + 0x5f, 0x43, 0x48, 0x49, 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, + 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x0b, 0x12, 0x33, 0x0a, 0x2f, 0x43, + 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x49, 0x47, 0x4e, + 0x41, 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, + 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x0c, + 0x12, 0x32, 0x0a, 0x2e, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x5f, 0x54, 0x59, 0x50, 0x45, + 0x5f, 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, + 0x5f, 0x53, 0x45, 0x41, 0x52, 0x43, 0x48, 0x5f, 0x41, 0x54, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, + 0x45, 0x53, 0x10, 0x0d, 0x12, 0x21, 0x0a, 0x1d, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x5f, 0x50, 0x52, 0x4f, 0x54, 0x4f, 0x43, 0x4f, 0x4c, 0x5f, 0x4d, 0x45, + 0x53, 0x53, 0x41, 0x47, 0x45, 0x10, 0x0e, 0x12, 0x2b, 0x0a, 0x27, 0x43, 0x4f, 0x4d, 0x4d, 0x41, + 0x4e, 0x44, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x59, 0x5f, 0x57, + 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x50, 0x52, 0x4f, 0x50, 0x45, 0x52, 0x54, 0x49, + 0x45, 0x53, 0x10, 0x10, 0x12, 0x29, 0x0a, 0x25, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x5f, 0x4e, 0x45, + 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x11, 0x12, + 0x2f, 0x0a, 0x2b, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, + 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x4e, + 0x45, 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x12, + 0x42, 0x88, 0x01, 0x0a, 0x18, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x42, 0x10, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x79, 0x70, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, + 0x01, 0x5a, 0x21, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, + 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x3b, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0xaa, 0x02, 0x17, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, + 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x56, 0x31, 0xea, 0x02, + 0x1a, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, + 0x3a, 0x3a, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_enums_v1_command_type_proto_rawDescOnce sync.Once + file_temporal_api_enums_v1_command_type_proto_rawDescData = file_temporal_api_enums_v1_command_type_proto_rawDesc +) + +func file_temporal_api_enums_v1_command_type_proto_rawDescGZIP() []byte { + file_temporal_api_enums_v1_command_type_proto_rawDescOnce.Do(func() { + file_temporal_api_enums_v1_command_type_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_enums_v1_command_type_proto_rawDescData) + }) + return file_temporal_api_enums_v1_command_type_proto_rawDescData +} + +var file_temporal_api_enums_v1_command_type_proto_enumTypes = make([]protoimpl.EnumInfo, 1) +var file_temporal_api_enums_v1_command_type_proto_goTypes = []any{ + (CommandType)(0), // 0: temporal.api.enums.v1.CommandType +} +var file_temporal_api_enums_v1_command_type_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_enums_v1_command_type_proto_init() } +func file_temporal_api_enums_v1_command_type_proto_init() { + if File_temporal_api_enums_v1_command_type_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_enums_v1_command_type_proto_rawDesc, + NumEnums: 1, + NumMessages: 0, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_enums_v1_command_type_proto_goTypes, + DependencyIndexes: file_temporal_api_enums_v1_command_type_proto_depIdxs, + EnumInfos: file_temporal_api_enums_v1_command_type_proto_enumTypes, + }.Build() + File_temporal_api_enums_v1_command_type_proto = out.File + file_temporal_api_enums_v1_command_type_proto_rawDesc = nil + file_temporal_api_enums_v1_command_type_proto_goTypes = nil + file_temporal_api_enums_v1_command_type_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/enums/v1/common.go-helpers.pb.go b/vendor/go.temporal.io/api/enums/v1/common.go-helpers.pb.go new file mode 100644 index 00000000000..b7fc50a7655 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/common.go-helpers.pb.go @@ -0,0 +1,155 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package enums + +import ( + "fmt" +) + +var ( + EncodingType_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Proto3": 1, + "Json": 2, + } +) + +// EncodingTypeFromString parses a EncodingType value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to EncodingType +func EncodingTypeFromString(s string) (EncodingType, error) { + if v, ok := EncodingType_value[s]; ok { + return EncodingType(v), nil + } else if v, ok := EncodingType_shorthandValue[s]; ok { + return EncodingType(v), nil + } + return EncodingType(0), fmt.Errorf("%s is not a valid EncodingType", s) +} + +var ( + IndexedValueType_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Text": 1, + "Keyword": 2, + "Int": 3, + "Double": 4, + "Bool": 5, + "Datetime": 6, + "KeywordList": 7, + } +) + +// IndexedValueTypeFromString parses a IndexedValueType value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to IndexedValueType +func IndexedValueTypeFromString(s string) (IndexedValueType, error) { + if v, ok := IndexedValueType_value[s]; ok { + return IndexedValueType(v), nil + } else if v, ok := IndexedValueType_shorthandValue[s]; ok { + return IndexedValueType(v), nil + } + return IndexedValueType(0), fmt.Errorf("%s is not a valid IndexedValueType", s) +} + +var ( + Severity_shorthandValue = map[string]int32{ + "Unspecified": 0, + "High": 1, + "Medium": 2, + "Low": 3, + } +) + +// SeverityFromString parses a Severity value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to Severity +func SeverityFromString(s string) (Severity, error) { + if v, ok := Severity_value[s]; ok { + return Severity(v), nil + } else if v, ok := Severity_shorthandValue[s]; ok { + return Severity(v), nil + } + return Severity(0), fmt.Errorf("%s is not a valid Severity", s) +} + +var ( + CallbackState_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Standby": 1, + "Scheduled": 2, + "BackingOff": 3, + "Failed": 4, + "Succeeded": 5, + } +) + +// CallbackStateFromString parses a CallbackState value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to CallbackState +func CallbackStateFromString(s string) (CallbackState, error) { + if v, ok := CallbackState_value[s]; ok { + return CallbackState(v), nil + } else if v, ok := CallbackState_shorthandValue[s]; ok { + return CallbackState(v), nil + } + return CallbackState(0), fmt.Errorf("%s is not a valid CallbackState", s) +} + +var ( + PendingNexusOperationState_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Scheduled": 1, + "BackingOff": 2, + "Started": 3, + } +) + +// PendingNexusOperationStateFromString parses a PendingNexusOperationState value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to PendingNexusOperationState +func PendingNexusOperationStateFromString(s string) (PendingNexusOperationState, error) { + if v, ok := PendingNexusOperationState_value[s]; ok { + return PendingNexusOperationState(v), nil + } else if v, ok := PendingNexusOperationState_shorthandValue[s]; ok { + return PendingNexusOperationState(v), nil + } + return PendingNexusOperationState(0), fmt.Errorf("%s is not a valid PendingNexusOperationState", s) +} + +var ( + NexusOperationCancellationState_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Scheduled": 1, + "BackingOff": 2, + "Succeeded": 3, + "Failed": 4, + "TimedOut": 5, + } +) + +// NexusOperationCancellationStateFromString parses a NexusOperationCancellationState value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to NexusOperationCancellationState +func NexusOperationCancellationStateFromString(s string) (NexusOperationCancellationState, error) { + if v, ok := NexusOperationCancellationState_value[s]; ok { + return NexusOperationCancellationState(v), nil + } else if v, ok := NexusOperationCancellationState_shorthandValue[s]; ok { + return NexusOperationCancellationState(v), nil + } + return NexusOperationCancellationState(0), fmt.Errorf("%s is not a valid NexusOperationCancellationState", s) +} diff --git a/vendor/go.temporal.io/api/enums/v1/common.pb.go b/vendor/go.temporal.io/api/enums/v1/common.pb.go new file mode 100644 index 00000000000..45be5275489 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/common.pb.go @@ -0,0 +1,631 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/enums/v1/common.proto + +package enums + +import ( + reflect "reflect" + "strconv" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type EncodingType int32 + +const ( + ENCODING_TYPE_UNSPECIFIED EncodingType = 0 + ENCODING_TYPE_PROTO3 EncodingType = 1 + ENCODING_TYPE_JSON EncodingType = 2 +) + +// Enum value maps for EncodingType. +var ( + EncodingType_name = map[int32]string{ + 0: "ENCODING_TYPE_UNSPECIFIED", + 1: "ENCODING_TYPE_PROTO3", + 2: "ENCODING_TYPE_JSON", + } + EncodingType_value = map[string]int32{ + "ENCODING_TYPE_UNSPECIFIED": 0, + "ENCODING_TYPE_PROTO3": 1, + "ENCODING_TYPE_JSON": 2, + } +) + +func (x EncodingType) Enum() *EncodingType { + p := new(EncodingType) + *p = x + return p +} + +func (x EncodingType) String() string { + switch x { + case ENCODING_TYPE_UNSPECIFIED: + return "Unspecified" + case ENCODING_TYPE_PROTO3: + return "Proto3" + case ENCODING_TYPE_JSON: + return "Json" + default: + return strconv.Itoa(int(x)) + } + +} + +func (EncodingType) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_common_proto_enumTypes[0].Descriptor() +} + +func (EncodingType) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_common_proto_enumTypes[0] +} + +func (x EncodingType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use EncodingType.Descriptor instead. +func (EncodingType) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_common_proto_rawDescGZIP(), []int{0} +} + +type IndexedValueType int32 + +const ( + INDEXED_VALUE_TYPE_UNSPECIFIED IndexedValueType = 0 + INDEXED_VALUE_TYPE_TEXT IndexedValueType = 1 + INDEXED_VALUE_TYPE_KEYWORD IndexedValueType = 2 + INDEXED_VALUE_TYPE_INT IndexedValueType = 3 + INDEXED_VALUE_TYPE_DOUBLE IndexedValueType = 4 + INDEXED_VALUE_TYPE_BOOL IndexedValueType = 5 + INDEXED_VALUE_TYPE_DATETIME IndexedValueType = 6 + INDEXED_VALUE_TYPE_KEYWORD_LIST IndexedValueType = 7 +) + +// Enum value maps for IndexedValueType. +var ( + IndexedValueType_name = map[int32]string{ + 0: "INDEXED_VALUE_TYPE_UNSPECIFIED", + 1: "INDEXED_VALUE_TYPE_TEXT", + 2: "INDEXED_VALUE_TYPE_KEYWORD", + 3: "INDEXED_VALUE_TYPE_INT", + 4: "INDEXED_VALUE_TYPE_DOUBLE", + 5: "INDEXED_VALUE_TYPE_BOOL", + 6: "INDEXED_VALUE_TYPE_DATETIME", + 7: "INDEXED_VALUE_TYPE_KEYWORD_LIST", + } + IndexedValueType_value = map[string]int32{ + "INDEXED_VALUE_TYPE_UNSPECIFIED": 0, + "INDEXED_VALUE_TYPE_TEXT": 1, + "INDEXED_VALUE_TYPE_KEYWORD": 2, + "INDEXED_VALUE_TYPE_INT": 3, + "INDEXED_VALUE_TYPE_DOUBLE": 4, + "INDEXED_VALUE_TYPE_BOOL": 5, + "INDEXED_VALUE_TYPE_DATETIME": 6, + "INDEXED_VALUE_TYPE_KEYWORD_LIST": 7, + } +) + +func (x IndexedValueType) Enum() *IndexedValueType { + p := new(IndexedValueType) + *p = x + return p +} + +func (x IndexedValueType) String() string { + switch x { + case INDEXED_VALUE_TYPE_UNSPECIFIED: + return "Unspecified" + case INDEXED_VALUE_TYPE_TEXT: + return "Text" + case INDEXED_VALUE_TYPE_KEYWORD: + return "Keyword" + case INDEXED_VALUE_TYPE_INT: + return "Int" + case INDEXED_VALUE_TYPE_DOUBLE: + return "Double" + case INDEXED_VALUE_TYPE_BOOL: + return "Bool" + case INDEXED_VALUE_TYPE_DATETIME: + return "Datetime" + case INDEXED_VALUE_TYPE_KEYWORD_LIST: + return "KeywordList" + default: + return strconv.Itoa(int(x)) + } + +} + +func (IndexedValueType) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_common_proto_enumTypes[1].Descriptor() +} + +func (IndexedValueType) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_common_proto_enumTypes[1] +} + +func (x IndexedValueType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use IndexedValueType.Descriptor instead. +func (IndexedValueType) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_common_proto_rawDescGZIP(), []int{1} +} + +type Severity int32 + +const ( + SEVERITY_UNSPECIFIED Severity = 0 + SEVERITY_HIGH Severity = 1 + SEVERITY_MEDIUM Severity = 2 + SEVERITY_LOW Severity = 3 +) + +// Enum value maps for Severity. +var ( + Severity_name = map[int32]string{ + 0: "SEVERITY_UNSPECIFIED", + 1: "SEVERITY_HIGH", + 2: "SEVERITY_MEDIUM", + 3: "SEVERITY_LOW", + } + Severity_value = map[string]int32{ + "SEVERITY_UNSPECIFIED": 0, + "SEVERITY_HIGH": 1, + "SEVERITY_MEDIUM": 2, + "SEVERITY_LOW": 3, + } +) + +func (x Severity) Enum() *Severity { + p := new(Severity) + *p = x + return p +} + +func (x Severity) String() string { + switch x { + case SEVERITY_UNSPECIFIED: + return "Unspecified" + case SEVERITY_HIGH: + return "High" + case SEVERITY_MEDIUM: + return "Medium" + case SEVERITY_LOW: + return "Low" + default: + return strconv.Itoa(int(x)) + } + +} + +func (Severity) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_common_proto_enumTypes[2].Descriptor() +} + +func (Severity) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_common_proto_enumTypes[2] +} + +func (x Severity) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use Severity.Descriptor instead. +func (Severity) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_common_proto_rawDescGZIP(), []int{2} +} + +// State of a callback. +type CallbackState int32 + +const ( + // Default value, unspecified state. + CALLBACK_STATE_UNSPECIFIED CallbackState = 0 + // Callback is standing by, waiting to be triggered. + CALLBACK_STATE_STANDBY CallbackState = 1 + // Callback is in the queue waiting to be executed or is currently executing. + CALLBACK_STATE_SCHEDULED CallbackState = 2 + // Callback has failed with a retryable error and is backing off before the next attempt. + CALLBACK_STATE_BACKING_OFF CallbackState = 3 + // Callback has failed. + CALLBACK_STATE_FAILED CallbackState = 4 + // Callback has succeeded. + CALLBACK_STATE_SUCCEEDED CallbackState = 5 +) + +// Enum value maps for CallbackState. +var ( + CallbackState_name = map[int32]string{ + 0: "CALLBACK_STATE_UNSPECIFIED", + 1: "CALLBACK_STATE_STANDBY", + 2: "CALLBACK_STATE_SCHEDULED", + 3: "CALLBACK_STATE_BACKING_OFF", + 4: "CALLBACK_STATE_FAILED", + 5: "CALLBACK_STATE_SUCCEEDED", + } + CallbackState_value = map[string]int32{ + "CALLBACK_STATE_UNSPECIFIED": 0, + "CALLBACK_STATE_STANDBY": 1, + "CALLBACK_STATE_SCHEDULED": 2, + "CALLBACK_STATE_BACKING_OFF": 3, + "CALLBACK_STATE_FAILED": 4, + "CALLBACK_STATE_SUCCEEDED": 5, + } +) + +func (x CallbackState) Enum() *CallbackState { + p := new(CallbackState) + *p = x + return p +} + +func (x CallbackState) String() string { + switch x { + case CALLBACK_STATE_UNSPECIFIED: + return "Unspecified" + case CALLBACK_STATE_STANDBY: + return "Standby" + case CALLBACK_STATE_SCHEDULED: + return "Scheduled" + case CALLBACK_STATE_BACKING_OFF: + return "BackingOff" + case CALLBACK_STATE_FAILED: + return "Failed" + case CALLBACK_STATE_SUCCEEDED: + return "Succeeded" + default: + return strconv.Itoa(int(x)) + } + +} + +func (CallbackState) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_common_proto_enumTypes[3].Descriptor() +} + +func (CallbackState) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_common_proto_enumTypes[3] +} + +func (x CallbackState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use CallbackState.Descriptor instead. +func (CallbackState) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_common_proto_rawDescGZIP(), []int{3} +} + +// State of a pending Nexus operation. +type PendingNexusOperationState int32 + +const ( + // Default value, unspecified state. + PENDING_NEXUS_OPERATION_STATE_UNSPECIFIED PendingNexusOperationState = 0 + // Operation is in the queue waiting to be executed or is currently executing. + PENDING_NEXUS_OPERATION_STATE_SCHEDULED PendingNexusOperationState = 1 + // Operation has failed with a retryable error and is backing off before the next attempt. + PENDING_NEXUS_OPERATION_STATE_BACKING_OFF PendingNexusOperationState = 2 + // Operation was started and will complete asynchronously. + PENDING_NEXUS_OPERATION_STATE_STARTED PendingNexusOperationState = 3 +) + +// Enum value maps for PendingNexusOperationState. +var ( + PendingNexusOperationState_name = map[int32]string{ + 0: "PENDING_NEXUS_OPERATION_STATE_UNSPECIFIED", + 1: "PENDING_NEXUS_OPERATION_STATE_SCHEDULED", + 2: "PENDING_NEXUS_OPERATION_STATE_BACKING_OFF", + 3: "PENDING_NEXUS_OPERATION_STATE_STARTED", + } + PendingNexusOperationState_value = map[string]int32{ + "PENDING_NEXUS_OPERATION_STATE_UNSPECIFIED": 0, + "PENDING_NEXUS_OPERATION_STATE_SCHEDULED": 1, + "PENDING_NEXUS_OPERATION_STATE_BACKING_OFF": 2, + "PENDING_NEXUS_OPERATION_STATE_STARTED": 3, + } +) + +func (x PendingNexusOperationState) Enum() *PendingNexusOperationState { + p := new(PendingNexusOperationState) + *p = x + return p +} + +func (x PendingNexusOperationState) String() string { + switch x { + case PENDING_NEXUS_OPERATION_STATE_UNSPECIFIED: + return "Unspecified" + case PENDING_NEXUS_OPERATION_STATE_SCHEDULED: + return "Scheduled" + case PENDING_NEXUS_OPERATION_STATE_BACKING_OFF: + return "BackingOff" + case PENDING_NEXUS_OPERATION_STATE_STARTED: + return "Started" + default: + return strconv.Itoa(int(x)) + } + +} + +func (PendingNexusOperationState) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_common_proto_enumTypes[4].Descriptor() +} + +func (PendingNexusOperationState) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_common_proto_enumTypes[4] +} + +func (x PendingNexusOperationState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use PendingNexusOperationState.Descriptor instead. +func (PendingNexusOperationState) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_common_proto_rawDescGZIP(), []int{4} +} + +// State of a Nexus operation cancellation. +type NexusOperationCancellationState int32 + +const ( + // Default value, unspecified state. + NEXUS_OPERATION_CANCELLATION_STATE_UNSPECIFIED NexusOperationCancellationState = 0 + // Cancellation request is in the queue waiting to be executed or is currently executing. + NEXUS_OPERATION_CANCELLATION_STATE_SCHEDULED NexusOperationCancellationState = 1 + // Cancellation request has failed with a retryable error and is backing off before the next attempt. + NEXUS_OPERATION_CANCELLATION_STATE_BACKING_OFF NexusOperationCancellationState = 2 + // Cancellation request succeeded. + NEXUS_OPERATION_CANCELLATION_STATE_SUCCEEDED NexusOperationCancellationState = 3 + // Cancellation request failed with a non-retryable error. + NEXUS_OPERATION_CANCELLATION_STATE_FAILED NexusOperationCancellationState = 4 + // The associated operation timed out - exceeded the user supplied schedule-to-close timeout. + NEXUS_OPERATION_CANCELLATION_STATE_TIMED_OUT NexusOperationCancellationState = 5 +) + +// Enum value maps for NexusOperationCancellationState. +var ( + NexusOperationCancellationState_name = map[int32]string{ + 0: "NEXUS_OPERATION_CANCELLATION_STATE_UNSPECIFIED", + 1: "NEXUS_OPERATION_CANCELLATION_STATE_SCHEDULED", + 2: "NEXUS_OPERATION_CANCELLATION_STATE_BACKING_OFF", + 3: "NEXUS_OPERATION_CANCELLATION_STATE_SUCCEEDED", + 4: "NEXUS_OPERATION_CANCELLATION_STATE_FAILED", + 5: "NEXUS_OPERATION_CANCELLATION_STATE_TIMED_OUT", + } + NexusOperationCancellationState_value = map[string]int32{ + "NEXUS_OPERATION_CANCELLATION_STATE_UNSPECIFIED": 0, + "NEXUS_OPERATION_CANCELLATION_STATE_SCHEDULED": 1, + "NEXUS_OPERATION_CANCELLATION_STATE_BACKING_OFF": 2, + "NEXUS_OPERATION_CANCELLATION_STATE_SUCCEEDED": 3, + "NEXUS_OPERATION_CANCELLATION_STATE_FAILED": 4, + "NEXUS_OPERATION_CANCELLATION_STATE_TIMED_OUT": 5, + } +) + +func (x NexusOperationCancellationState) Enum() *NexusOperationCancellationState { + p := new(NexusOperationCancellationState) + *p = x + return p +} + +func (x NexusOperationCancellationState) String() string { + switch x { + case NEXUS_OPERATION_CANCELLATION_STATE_UNSPECIFIED: + return "Unspecified" + case NEXUS_OPERATION_CANCELLATION_STATE_SCHEDULED: + return "Scheduled" + case NEXUS_OPERATION_CANCELLATION_STATE_BACKING_OFF: + return "BackingOff" + case NEXUS_OPERATION_CANCELLATION_STATE_SUCCEEDED: + return "Succeeded" + case NEXUS_OPERATION_CANCELLATION_STATE_FAILED: + return "Failed" + case NEXUS_OPERATION_CANCELLATION_STATE_TIMED_OUT: + return "TimedOut" + default: + return strconv.Itoa(int(x)) + } + +} + +func (NexusOperationCancellationState) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_common_proto_enumTypes[5].Descriptor() +} + +func (NexusOperationCancellationState) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_common_proto_enumTypes[5] +} + +func (x NexusOperationCancellationState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use NexusOperationCancellationState.Descriptor instead. +func (NexusOperationCancellationState) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_common_proto_rawDescGZIP(), []int{5} +} + +var File_temporal_api_enums_v1_common_proto protoreflect.FileDescriptor + +var file_temporal_api_enums_v1_common_proto_rawDesc = []byte{ + 0x0a, 0x22, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2a, 0x5f, 0x0a, 0x0c, 0x45, + 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x19, 0x45, + 0x4e, 0x43, 0x4f, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, + 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x18, 0x0a, 0x14, 0x45, 0x4e, + 0x43, 0x4f, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x50, 0x52, 0x4f, 0x54, + 0x4f, 0x33, 0x10, 0x01, 0x12, 0x16, 0x0a, 0x12, 0x45, 0x4e, 0x43, 0x4f, 0x44, 0x49, 0x4e, 0x47, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4a, 0x53, 0x4f, 0x4e, 0x10, 0x02, 0x2a, 0x91, 0x02, 0x0a, + 0x10, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x54, 0x79, 0x70, + 0x65, 0x12, 0x22, 0x0a, 0x1e, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x45, 0x44, 0x5f, 0x56, 0x41, 0x4c, + 0x55, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, + 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x1b, 0x0a, 0x17, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x45, 0x44, + 0x5f, 0x56, 0x41, 0x4c, 0x55, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x54, 0x45, 0x58, 0x54, + 0x10, 0x01, 0x12, 0x1e, 0x0a, 0x1a, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x45, 0x44, 0x5f, 0x56, 0x41, + 0x4c, 0x55, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4b, 0x45, 0x59, 0x57, 0x4f, 0x52, 0x44, + 0x10, 0x02, 0x12, 0x1a, 0x0a, 0x16, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x45, 0x44, 0x5f, 0x56, 0x41, + 0x4c, 0x55, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x10, 0x03, 0x12, 0x1d, + 0x0a, 0x19, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x45, 0x44, 0x5f, 0x56, 0x41, 0x4c, 0x55, 0x45, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x04, 0x12, 0x1b, 0x0a, + 0x17, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x45, 0x44, 0x5f, 0x56, 0x41, 0x4c, 0x55, 0x45, 0x5f, 0x54, + 0x59, 0x50, 0x45, 0x5f, 0x42, 0x4f, 0x4f, 0x4c, 0x10, 0x05, 0x12, 0x1f, 0x0a, 0x1b, 0x49, 0x4e, + 0x44, 0x45, 0x58, 0x45, 0x44, 0x5f, 0x56, 0x41, 0x4c, 0x55, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, + 0x5f, 0x44, 0x41, 0x54, 0x45, 0x54, 0x49, 0x4d, 0x45, 0x10, 0x06, 0x12, 0x23, 0x0a, 0x1f, 0x49, + 0x4e, 0x44, 0x45, 0x58, 0x45, 0x44, 0x5f, 0x56, 0x41, 0x4c, 0x55, 0x45, 0x5f, 0x54, 0x59, 0x50, + 0x45, 0x5f, 0x4b, 0x45, 0x59, 0x57, 0x4f, 0x52, 0x44, 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x10, 0x07, + 0x2a, 0x5e, 0x0a, 0x08, 0x53, 0x65, 0x76, 0x65, 0x72, 0x69, 0x74, 0x79, 0x12, 0x18, 0x0a, 0x14, + 0x53, 0x45, 0x56, 0x45, 0x52, 0x49, 0x54, 0x59, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, + 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x11, 0x0a, 0x0d, 0x53, 0x45, 0x56, 0x45, 0x52, 0x49, + 0x54, 0x59, 0x5f, 0x48, 0x49, 0x47, 0x48, 0x10, 0x01, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x45, 0x56, + 0x45, 0x52, 0x49, 0x54, 0x59, 0x5f, 0x4d, 0x45, 0x44, 0x49, 0x55, 0x4d, 0x10, 0x02, 0x12, 0x10, + 0x0a, 0x0c, 0x53, 0x45, 0x56, 0x45, 0x52, 0x49, 0x54, 0x59, 0x5f, 0x4c, 0x4f, 0x57, 0x10, 0x03, + 0x2a, 0xc2, 0x01, 0x0a, 0x0d, 0x43, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x1e, 0x0a, 0x1a, 0x43, 0x41, 0x4c, 0x4c, 0x42, 0x41, 0x43, 0x4b, 0x5f, 0x53, + 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, + 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x43, 0x41, 0x4c, 0x4c, 0x42, 0x41, 0x43, 0x4b, 0x5f, 0x53, + 0x54, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x4e, 0x44, 0x42, 0x59, 0x10, 0x01, 0x12, 0x1c, + 0x0a, 0x18, 0x43, 0x41, 0x4c, 0x4c, 0x42, 0x41, 0x43, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, + 0x5f, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a, + 0x43, 0x41, 0x4c, 0x4c, 0x42, 0x41, 0x43, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x42, + 0x41, 0x43, 0x4b, 0x49, 0x4e, 0x47, 0x5f, 0x4f, 0x46, 0x46, 0x10, 0x03, 0x12, 0x19, 0x0a, 0x15, + 0x43, 0x41, 0x4c, 0x4c, 0x42, 0x41, 0x43, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x46, + 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x04, 0x12, 0x1c, 0x0a, 0x18, 0x43, 0x41, 0x4c, 0x4c, 0x42, + 0x41, 0x43, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x55, 0x43, 0x43, 0x45, 0x45, + 0x44, 0x45, 0x44, 0x10, 0x05, 0x2a, 0xd2, 0x01, 0x0a, 0x1a, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, + 0x67, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x12, 0x2d, 0x0a, 0x29, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x5f, + 0x4e, 0x45, 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, + 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, + 0x44, 0x10, 0x00, 0x12, 0x2b, 0x0a, 0x27, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x4e, + 0x45, 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, + 0x54, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x44, 0x10, 0x01, + 0x12, 0x2d, 0x0a, 0x29, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x4e, 0x45, 0x58, 0x55, + 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, + 0x45, 0x5f, 0x42, 0x41, 0x43, 0x4b, 0x49, 0x4e, 0x47, 0x5f, 0x4f, 0x46, 0x46, 0x10, 0x02, 0x12, + 0x29, 0x0a, 0x25, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x4e, 0x45, 0x58, 0x55, 0x53, + 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, + 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, 0x10, 0x03, 0x2a, 0xce, 0x02, 0x0a, 0x1f, 0x4e, + 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, + 0x63, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x32, + 0x0a, 0x2e, 0x4e, 0x45, 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, + 0x4e, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, + 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, + 0x10, 0x00, 0x12, 0x30, 0x0a, 0x2c, 0x4e, 0x45, 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, + 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x41, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, + 0x45, 0x44, 0x10, 0x01, 0x12, 0x32, 0x0a, 0x2e, 0x4e, 0x45, 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, + 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x41, + 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x42, 0x41, 0x43, 0x4b, 0x49, + 0x4e, 0x47, 0x5f, 0x4f, 0x46, 0x46, 0x10, 0x02, 0x12, 0x30, 0x0a, 0x2c, 0x4e, 0x45, 0x58, 0x55, + 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x41, 0x4e, 0x43, + 0x45, 0x4c, 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x53, + 0x55, 0x43, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x03, 0x12, 0x2d, 0x0a, 0x29, 0x4e, 0x45, + 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x41, + 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, + 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x04, 0x12, 0x30, 0x0a, 0x2c, 0x4e, 0x45, 0x58, + 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x41, 0x4e, + 0x43, 0x45, 0x4c, 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, + 0x54, 0x49, 0x4d, 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x05, 0x42, 0x83, 0x01, 0x0a, 0x18, + 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x42, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x21, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, + 0x73, 0x2f, 0x76, 0x31, 0x3b, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0xaa, 0x02, 0x17, 0x54, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x45, 0x6e, 0x75, 0x6d, + 0x73, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1a, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, + 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x3a, 0x3a, 0x56, + 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_enums_v1_common_proto_rawDescOnce sync.Once + file_temporal_api_enums_v1_common_proto_rawDescData = file_temporal_api_enums_v1_common_proto_rawDesc +) + +func file_temporal_api_enums_v1_common_proto_rawDescGZIP() []byte { + file_temporal_api_enums_v1_common_proto_rawDescOnce.Do(func() { + file_temporal_api_enums_v1_common_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_enums_v1_common_proto_rawDescData) + }) + return file_temporal_api_enums_v1_common_proto_rawDescData +} + +var file_temporal_api_enums_v1_common_proto_enumTypes = make([]protoimpl.EnumInfo, 6) +var file_temporal_api_enums_v1_common_proto_goTypes = []any{ + (EncodingType)(0), // 0: temporal.api.enums.v1.EncodingType + (IndexedValueType)(0), // 1: temporal.api.enums.v1.IndexedValueType + (Severity)(0), // 2: temporal.api.enums.v1.Severity + (CallbackState)(0), // 3: temporal.api.enums.v1.CallbackState + (PendingNexusOperationState)(0), // 4: temporal.api.enums.v1.PendingNexusOperationState + (NexusOperationCancellationState)(0), // 5: temporal.api.enums.v1.NexusOperationCancellationState +} +var file_temporal_api_enums_v1_common_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_enums_v1_common_proto_init() } +func file_temporal_api_enums_v1_common_proto_init() { + if File_temporal_api_enums_v1_common_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_enums_v1_common_proto_rawDesc, + NumEnums: 6, + NumMessages: 0, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_enums_v1_common_proto_goTypes, + DependencyIndexes: file_temporal_api_enums_v1_common_proto_depIdxs, + EnumInfos: file_temporal_api_enums_v1_common_proto_enumTypes, + }.Build() + File_temporal_api_enums_v1_common_proto = out.File + file_temporal_api_enums_v1_common_proto_rawDesc = nil + file_temporal_api_enums_v1_common_proto_goTypes = nil + file_temporal_api_enums_v1_common_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/enums/v1/event_type.go-helpers.pb.go b/vendor/go.temporal.io/api/enums/v1/event_type.go-helpers.pb.go new file mode 100644 index 00000000000..5f505f41b6f --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/event_type.go-helpers.pb.go @@ -0,0 +1,99 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package enums + +import ( + "fmt" +) + +var ( + EventType_shorthandValue = map[string]int32{ + "Unspecified": 0, + "WorkflowExecutionStarted": 1, + "WorkflowExecutionCompleted": 2, + "WorkflowExecutionFailed": 3, + "WorkflowExecutionTimedOut": 4, + "WorkflowTaskScheduled": 5, + "WorkflowTaskStarted": 6, + "WorkflowTaskCompleted": 7, + "WorkflowTaskTimedOut": 8, + "WorkflowTaskFailed": 9, + "ActivityTaskScheduled": 10, + "ActivityTaskStarted": 11, + "ActivityTaskCompleted": 12, + "ActivityTaskFailed": 13, + "ActivityTaskTimedOut": 14, + "ActivityTaskCancelRequested": 15, + "ActivityTaskCanceled": 16, + "TimerStarted": 17, + "TimerFired": 18, + "TimerCanceled": 19, + "WorkflowExecutionCancelRequested": 20, + "WorkflowExecutionCanceled": 21, + "RequestCancelExternalWorkflowExecutionInitiated": 22, + "RequestCancelExternalWorkflowExecutionFailed": 23, + "ExternalWorkflowExecutionCancelRequested": 24, + "MarkerRecorded": 25, + "WorkflowExecutionSignaled": 26, + "WorkflowExecutionTerminated": 27, + "WorkflowExecutionContinuedAsNew": 28, + "StartChildWorkflowExecutionInitiated": 29, + "StartChildWorkflowExecutionFailed": 30, + "ChildWorkflowExecutionStarted": 31, + "ChildWorkflowExecutionCompleted": 32, + "ChildWorkflowExecutionFailed": 33, + "ChildWorkflowExecutionCanceled": 34, + "ChildWorkflowExecutionTimedOut": 35, + "ChildWorkflowExecutionTerminated": 36, + "SignalExternalWorkflowExecutionInitiated": 37, + "SignalExternalWorkflowExecutionFailed": 38, + "ExternalWorkflowExecutionSignaled": 39, + "UpsertWorkflowSearchAttributes": 40, + "WorkflowExecutionUpdateAdmitted": 47, + "WorkflowExecutionUpdateAccepted": 41, + "WorkflowExecutionUpdateRejected": 42, + "WorkflowExecutionUpdateCompleted": 43, + "WorkflowPropertiesModifiedExternally": 44, + "ActivityPropertiesModifiedExternally": 45, + "WorkflowPropertiesModified": 46, + "NexusOperationScheduled": 48, + "NexusOperationStarted": 49, + "NexusOperationCompleted": 50, + "NexusOperationFailed": 51, + "NexusOperationCanceled": 52, + "NexusOperationTimedOut": 53, + "NexusOperationCancelRequested": 54, + } +) + +// EventTypeFromString parses a EventType value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to EventType +func EventTypeFromString(s string) (EventType, error) { + if v, ok := EventType_value[s]; ok { + return EventType(v), nil + } else if v, ok := EventType_shorthandValue[s]; ok { + return EventType(v), nil + } + return EventType(0), fmt.Errorf("%s is not a valid EventType", s) +} diff --git a/vendor/go.temporal.io/api/enums/v1/event_type.pb.go b/vendor/go.temporal.io/api/enums/v1/event_type.pb.go new file mode 100644 index 00000000000..c510afbba25 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/event_type.pb.go @@ -0,0 +1,689 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/enums/v1/event_type.proto + +package enums + +import ( + reflect "reflect" + "strconv" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// Whenever this list of events is changed do change the function shouldBufferEvent in mutableStateBuilder.go to make sure to do the correct event ordering +type EventType int32 + +const ( + // Place holder and should never appear in a Workflow execution history + EVENT_TYPE_UNSPECIFIED EventType = 0 + // Workflow execution has been triggered/started + // It contains Workflow execution inputs, as well as Workflow timeout configurations + EVENT_TYPE_WORKFLOW_EXECUTION_STARTED EventType = 1 + // Workflow execution has successfully completed and contains Workflow execution results + EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED EventType = 2 + // Workflow execution has unsuccessfully completed and contains the Workflow execution error + EVENT_TYPE_WORKFLOW_EXECUTION_FAILED EventType = 3 + // Workflow execution has timed out by the Temporal Server + // Usually due to the Workflow having not been completed within timeout settings + EVENT_TYPE_WORKFLOW_EXECUTION_TIMED_OUT EventType = 4 + // Workflow Task has been scheduled and the SDK client should now be able to process any new history events + EVENT_TYPE_WORKFLOW_TASK_SCHEDULED EventType = 5 + // Workflow Task has started and the SDK client has picked up the Workflow Task and is processing new history events + EVENT_TYPE_WORKFLOW_TASK_STARTED EventType = 6 + // Workflow Task has completed + // The SDK client picked up the Workflow Task and processed new history events + // SDK client may or may not ask the Temporal Server to do additional work, such as: + // EVENT_TYPE_ACTIVITY_TASK_SCHEDULED + // EVENT_TYPE_TIMER_STARTED + // EVENT_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES + // EVENT_TYPE_MARKER_RECORDED + // EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_INITIATED + // EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED + // EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED + // EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED + // EVENT_TYPE_WORKFLOW_EXECUTION_FAILED + // EVENT_TYPE_WORKFLOW_EXECUTION_CANCELED + // EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW + EVENT_TYPE_WORKFLOW_TASK_COMPLETED EventType = 7 + // Workflow Task encountered a timeout + // Either an SDK client with a local cache was not available at the time, or it took too long for the SDK client to process the task + EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT EventType = 8 + // Workflow Task encountered a failure + // Usually this means that the Workflow was non-deterministic + // However, the Workflow reset functionality also uses this event + EVENT_TYPE_WORKFLOW_TASK_FAILED EventType = 9 + // Activity Task was scheduled + // The SDK client should pick up this activity task and execute + // This event type contains activity inputs, as well as activity timeout configurations + EVENT_TYPE_ACTIVITY_TASK_SCHEDULED EventType = 10 + // Activity Task has started executing + // The SDK client has picked up the Activity Task and is processing the Activity invocation + EVENT_TYPE_ACTIVITY_TASK_STARTED EventType = 11 + // Activity Task has finished successfully + // The SDK client has picked up and successfully completed the Activity Task + // This event type contains Activity execution results + EVENT_TYPE_ACTIVITY_TASK_COMPLETED EventType = 12 + // Activity Task has finished unsuccessfully + // The SDK picked up the Activity Task but unsuccessfully completed it + // This event type contains Activity execution errors + EVENT_TYPE_ACTIVITY_TASK_FAILED EventType = 13 + // Activity has timed out according to the Temporal Server + // Activity did not complete within the timeout settings + EVENT_TYPE_ACTIVITY_TASK_TIMED_OUT EventType = 14 + // A request to cancel the Activity has occurred + // The SDK client will be able to confirm cancellation of an Activity during an Activity heartbeat + EVENT_TYPE_ACTIVITY_TASK_CANCEL_REQUESTED EventType = 15 + // Activity has been cancelled + EVENT_TYPE_ACTIVITY_TASK_CANCELED EventType = 16 + // A timer has started + EVENT_TYPE_TIMER_STARTED EventType = 17 + // A timer has fired + EVENT_TYPE_TIMER_FIRED EventType = 18 + // A time has been cancelled + EVENT_TYPE_TIMER_CANCELED EventType = 19 + // A request has been made to cancel the Workflow execution + EVENT_TYPE_WORKFLOW_EXECUTION_CANCEL_REQUESTED EventType = 20 + // SDK client has confirmed the cancellation request and the Workflow execution has been cancelled + EVENT_TYPE_WORKFLOW_EXECUTION_CANCELED EventType = 21 + // Workflow has requested that the Temporal Server try to cancel another Workflow + EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED EventType = 22 + // Temporal Server could not cancel the targeted Workflow + // This is usually because the target Workflow could not be found + EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED EventType = 23 + // Temporal Server has successfully requested the cancellation of the target Workflow + EVENT_TYPE_EXTERNAL_WORKFLOW_EXECUTION_CANCEL_REQUESTED EventType = 24 + // A marker has been recorded. + // This event type is transparent to the Temporal Server + // The Server will only store it and will not try to understand it. + EVENT_TYPE_MARKER_RECORDED EventType = 25 + // Workflow has received a Signal event + // The event type contains the Signal name, as well as a Signal payload + EVENT_TYPE_WORKFLOW_EXECUTION_SIGNALED EventType = 26 + // Workflow execution has been forcefully terminated + // This is usually because the terminate Workflow API was called + EVENT_TYPE_WORKFLOW_EXECUTION_TERMINATED EventType = 27 + // Workflow has successfully completed and a new Workflow has been started within the same transaction + // Contains last Workflow execution results as well as new Workflow execution inputs + EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW EventType = 28 + // Temporal Server will try to start a child Workflow + EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_INITIATED EventType = 29 + // Child Workflow execution cannot be started/triggered + // Usually due to a child Workflow ID collision + EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_FAILED EventType = 30 + // Child Workflow execution has successfully started/triggered + EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_STARTED EventType = 31 + // Child Workflow execution has successfully completed + EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_COMPLETED EventType = 32 + // Child Workflow execution has unsuccessfully completed + EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_FAILED EventType = 33 + // Child Workflow execution has been cancelled + EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_CANCELED EventType = 34 + // Child Workflow execution has timed out by the Temporal Server + EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_TIMED_OUT EventType = 35 + // Child Workflow execution has been terminated + EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_TERMINATED EventType = 36 + // Temporal Server will try to Signal the targeted Workflow + // Contains the Signal name, as well as a Signal payload + EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED EventType = 37 + // Temporal Server cannot Signal the targeted Workflow + // Usually because the Workflow could not be found + EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED EventType = 38 + // Temporal Server has successfully Signaled the targeted Workflow + EVENT_TYPE_EXTERNAL_WORKFLOW_EXECUTION_SIGNALED EventType = 39 + // Workflow search attributes should be updated and synchronized with the visibility store + EVENT_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES EventType = 40 + // An update was admitted. Note that not all admitted updates result in this + // event. See UpdateAdmittedEventOrigin for situations in which this event + // is created. + EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_ADMITTED EventType = 47 + // An update was accepted (i.e. passed validation, perhaps because no validator was defined) + EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_ACCEPTED EventType = 41 + // This event is never written to history. + EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_REJECTED EventType = 42 + // An update completed + EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_COMPLETED EventType = 43 + // Some property or properties of the workflow as a whole have changed by non-workflow code. + // The distinction of external vs. command-based modification is important so the SDK can + // maintain determinism when using the command-based approach. + EVENT_TYPE_WORKFLOW_PROPERTIES_MODIFIED_EXTERNALLY EventType = 44 + // Some property or properties of an already-scheduled activity have changed by non-workflow code. + // The distinction of external vs. command-based modification is important so the SDK can + // maintain determinism when using the command-based approach. + EVENT_TYPE_ACTIVITY_PROPERTIES_MODIFIED_EXTERNALLY EventType = 45 + // Workflow properties modified by user workflow code + EVENT_TYPE_WORKFLOW_PROPERTIES_MODIFIED EventType = 46 + // A Nexus operation was scheduled using a ScheduleNexusOperation command. + EVENT_TYPE_NEXUS_OPERATION_SCHEDULED EventType = 48 + // An asynchronous Nexus operation was started by a Nexus handler. + EVENT_TYPE_NEXUS_OPERATION_STARTED EventType = 49 + // A Nexus operation completed successfully. + EVENT_TYPE_NEXUS_OPERATION_COMPLETED EventType = 50 + // A Nexus operation failed. + EVENT_TYPE_NEXUS_OPERATION_FAILED EventType = 51 + // A Nexus operation completed as canceled. + EVENT_TYPE_NEXUS_OPERATION_CANCELED EventType = 52 + // A Nexus operation timed out. + EVENT_TYPE_NEXUS_OPERATION_TIMED_OUT EventType = 53 + // A Nexus operation was requested to be canceled using a RequestCancelNexusOperation command. + EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUESTED EventType = 54 +) + +// Enum value maps for EventType. +var ( + EventType_name = map[int32]string{ + 0: "EVENT_TYPE_UNSPECIFIED", + 1: "EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", + 2: "EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED", + 3: "EVENT_TYPE_WORKFLOW_EXECUTION_FAILED", + 4: "EVENT_TYPE_WORKFLOW_EXECUTION_TIMED_OUT", + 5: "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + 6: "EVENT_TYPE_WORKFLOW_TASK_STARTED", + 7: "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + 8: "EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT", + 9: "EVENT_TYPE_WORKFLOW_TASK_FAILED", + 10: "EVENT_TYPE_ACTIVITY_TASK_SCHEDULED", + 11: "EVENT_TYPE_ACTIVITY_TASK_STARTED", + 12: "EVENT_TYPE_ACTIVITY_TASK_COMPLETED", + 13: "EVENT_TYPE_ACTIVITY_TASK_FAILED", + 14: "EVENT_TYPE_ACTIVITY_TASK_TIMED_OUT", + 15: "EVENT_TYPE_ACTIVITY_TASK_CANCEL_REQUESTED", + 16: "EVENT_TYPE_ACTIVITY_TASK_CANCELED", + 17: "EVENT_TYPE_TIMER_STARTED", + 18: "EVENT_TYPE_TIMER_FIRED", + 19: "EVENT_TYPE_TIMER_CANCELED", + 20: "EVENT_TYPE_WORKFLOW_EXECUTION_CANCEL_REQUESTED", + 21: "EVENT_TYPE_WORKFLOW_EXECUTION_CANCELED", + 22: "EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED", + 23: "EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED", + 24: "EVENT_TYPE_EXTERNAL_WORKFLOW_EXECUTION_CANCEL_REQUESTED", + 25: "EVENT_TYPE_MARKER_RECORDED", + 26: "EVENT_TYPE_WORKFLOW_EXECUTION_SIGNALED", + 27: "EVENT_TYPE_WORKFLOW_EXECUTION_TERMINATED", + 28: "EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW", + 29: "EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_INITIATED", + 30: "EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_FAILED", + 31: "EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_STARTED", + 32: "EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_COMPLETED", + 33: "EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_FAILED", + 34: "EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_CANCELED", + 35: "EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_TIMED_OUT", + 36: "EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_TERMINATED", + 37: "EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED", + 38: "EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED", + 39: "EVENT_TYPE_EXTERNAL_WORKFLOW_EXECUTION_SIGNALED", + 40: "EVENT_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES", + 47: "EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_ADMITTED", + 41: "EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_ACCEPTED", + 42: "EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_REJECTED", + 43: "EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_COMPLETED", + 44: "EVENT_TYPE_WORKFLOW_PROPERTIES_MODIFIED_EXTERNALLY", + 45: "EVENT_TYPE_ACTIVITY_PROPERTIES_MODIFIED_EXTERNALLY", + 46: "EVENT_TYPE_WORKFLOW_PROPERTIES_MODIFIED", + 48: "EVENT_TYPE_NEXUS_OPERATION_SCHEDULED", + 49: "EVENT_TYPE_NEXUS_OPERATION_STARTED", + 50: "EVENT_TYPE_NEXUS_OPERATION_COMPLETED", + 51: "EVENT_TYPE_NEXUS_OPERATION_FAILED", + 52: "EVENT_TYPE_NEXUS_OPERATION_CANCELED", + 53: "EVENT_TYPE_NEXUS_OPERATION_TIMED_OUT", + 54: "EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUESTED", + } + EventType_value = map[string]int32{ + "EVENT_TYPE_UNSPECIFIED": 0, + "EVENT_TYPE_WORKFLOW_EXECUTION_STARTED": 1, + "EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED": 2, + "EVENT_TYPE_WORKFLOW_EXECUTION_FAILED": 3, + "EVENT_TYPE_WORKFLOW_EXECUTION_TIMED_OUT": 4, + "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED": 5, + "EVENT_TYPE_WORKFLOW_TASK_STARTED": 6, + "EVENT_TYPE_WORKFLOW_TASK_COMPLETED": 7, + "EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT": 8, + "EVENT_TYPE_WORKFLOW_TASK_FAILED": 9, + "EVENT_TYPE_ACTIVITY_TASK_SCHEDULED": 10, + "EVENT_TYPE_ACTIVITY_TASK_STARTED": 11, + "EVENT_TYPE_ACTIVITY_TASK_COMPLETED": 12, + "EVENT_TYPE_ACTIVITY_TASK_FAILED": 13, + "EVENT_TYPE_ACTIVITY_TASK_TIMED_OUT": 14, + "EVENT_TYPE_ACTIVITY_TASK_CANCEL_REQUESTED": 15, + "EVENT_TYPE_ACTIVITY_TASK_CANCELED": 16, + "EVENT_TYPE_TIMER_STARTED": 17, + "EVENT_TYPE_TIMER_FIRED": 18, + "EVENT_TYPE_TIMER_CANCELED": 19, + "EVENT_TYPE_WORKFLOW_EXECUTION_CANCEL_REQUESTED": 20, + "EVENT_TYPE_WORKFLOW_EXECUTION_CANCELED": 21, + "EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED": 22, + "EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED": 23, + "EVENT_TYPE_EXTERNAL_WORKFLOW_EXECUTION_CANCEL_REQUESTED": 24, + "EVENT_TYPE_MARKER_RECORDED": 25, + "EVENT_TYPE_WORKFLOW_EXECUTION_SIGNALED": 26, + "EVENT_TYPE_WORKFLOW_EXECUTION_TERMINATED": 27, + "EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW": 28, + "EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_INITIATED": 29, + "EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_FAILED": 30, + "EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_STARTED": 31, + "EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_COMPLETED": 32, + "EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_FAILED": 33, + "EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_CANCELED": 34, + "EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_TIMED_OUT": 35, + "EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_TERMINATED": 36, + "EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED": 37, + "EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED": 38, + "EVENT_TYPE_EXTERNAL_WORKFLOW_EXECUTION_SIGNALED": 39, + "EVENT_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES": 40, + "EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_ADMITTED": 47, + "EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_ACCEPTED": 41, + "EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_REJECTED": 42, + "EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_COMPLETED": 43, + "EVENT_TYPE_WORKFLOW_PROPERTIES_MODIFIED_EXTERNALLY": 44, + "EVENT_TYPE_ACTIVITY_PROPERTIES_MODIFIED_EXTERNALLY": 45, + "EVENT_TYPE_WORKFLOW_PROPERTIES_MODIFIED": 46, + "EVENT_TYPE_NEXUS_OPERATION_SCHEDULED": 48, + "EVENT_TYPE_NEXUS_OPERATION_STARTED": 49, + "EVENT_TYPE_NEXUS_OPERATION_COMPLETED": 50, + "EVENT_TYPE_NEXUS_OPERATION_FAILED": 51, + "EVENT_TYPE_NEXUS_OPERATION_CANCELED": 52, + "EVENT_TYPE_NEXUS_OPERATION_TIMED_OUT": 53, + "EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUESTED": 54, + } +) + +func (x EventType) Enum() *EventType { + p := new(EventType) + *p = x + return p +} + +func (x EventType) String() string { + switch x { + case EVENT_TYPE_UNSPECIFIED: + return "Unspecified" + case EVENT_TYPE_WORKFLOW_EXECUTION_STARTED: + return "WorkflowExecutionStarted" + case EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED: + return "WorkflowExecutionCompleted" + case EVENT_TYPE_WORKFLOW_EXECUTION_FAILED: + return "WorkflowExecutionFailed" + case EVENT_TYPE_WORKFLOW_EXECUTION_TIMED_OUT: + return "WorkflowExecutionTimedOut" + case EVENT_TYPE_WORKFLOW_TASK_SCHEDULED: + return "WorkflowTaskScheduled" + case EVENT_TYPE_WORKFLOW_TASK_STARTED: + + // Deprecated: Use EventType.Descriptor instead. + return "WorkflowTaskStarted" + case EVENT_TYPE_WORKFLOW_TASK_COMPLETED: + return "WorkflowTaskCompleted" + case EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT: + return "WorkflowTaskTimedOut" + case EVENT_TYPE_WORKFLOW_TASK_FAILED: + return "WorkflowTaskFailed" + case EVENT_TYPE_ACTIVITY_TASK_SCHEDULED: + return "ActivityTaskScheduled" + case EVENT_TYPE_ACTIVITY_TASK_STARTED: + return "ActivityTaskStarted" + case EVENT_TYPE_ACTIVITY_TASK_COMPLETED: + return "ActivityTaskCompleted" + case EVENT_TYPE_ACTIVITY_TASK_FAILED: + return "ActivityTaskFailed" + case EVENT_TYPE_ACTIVITY_TASK_TIMED_OUT: + return "ActivityTaskTimedOut" + case EVENT_TYPE_ACTIVITY_TASK_CANCEL_REQUESTED: + return "ActivityTaskCancelRequested" + case EVENT_TYPE_ACTIVITY_TASK_CANCELED: + return "ActivityTaskCanceled" + case EVENT_TYPE_TIMER_STARTED: + return "TimerStarted" + case EVENT_TYPE_TIMER_FIRED: + return "TimerFired" + case EVENT_TYPE_TIMER_CANCELED: + return "TimerCanceled" + case EVENT_TYPE_WORKFLOW_EXECUTION_CANCEL_REQUESTED: + return "WorkflowExecutionCancelRequested" + case EVENT_TYPE_WORKFLOW_EXECUTION_CANCELED: + return "WorkflowExecutionCanceled" + case EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED: + return "RequestCancelExternalWorkflowExecutionInitiated" + case EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED: + return "RequestCancelExternalWorkflowExecutionFailed" + case EVENT_TYPE_EXTERNAL_WORKFLOW_EXECUTION_CANCEL_REQUESTED: + return "ExternalWorkflowExecutionCancelRequested" + case EVENT_TYPE_MARKER_RECORDED: + return "MarkerRecorded" + case EVENT_TYPE_WORKFLOW_EXECUTION_SIGNALED: + return "WorkflowExecutionSignaled" + case EVENT_TYPE_WORKFLOW_EXECUTION_TERMINATED: + return "WorkflowExecutionTerminated" + case EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW: + return "WorkflowExecutionContinuedAsNew" + case EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_INITIATED: + return "StartChildWorkflowExecutionInitiated" + case EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_FAILED: + return "StartChildWorkflowExecutionFailed" + case EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_STARTED: + return "ChildWorkflowExecutionStarted" + case EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_COMPLETED: + return "ChildWorkflowExecutionCompleted" + case EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_FAILED: + return "ChildWorkflowExecutionFailed" + case EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_CANCELED: + return "ChildWorkflowExecutionCanceled" + case EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_TIMED_OUT: + return "ChildWorkflowExecutionTimedOut" + case EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_TERMINATED: + return "ChildWorkflowExecutionTerminated" + case EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED: + return "SignalExternalWorkflowExecutionInitiated" + case EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED: + return "SignalExternalWorkflowExecutionFailed" + case EVENT_TYPE_EXTERNAL_WORKFLOW_EXECUTION_SIGNALED: + return "ExternalWorkflowExecutionSignaled" + case EVENT_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES: + return "UpsertWorkflowSearchAttributes" + case EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_ADMITTED: + return "WorkflowExecutionUpdateAdmitted" + case EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_ACCEPTED: + return "WorkflowExecutionUpdateAccepted" + case EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_REJECTED: + return "WorkflowExecutionUpdateRejected" + case EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_COMPLETED: + return "WorkflowExecutionUpdateCompleted" + case EVENT_TYPE_WORKFLOW_PROPERTIES_MODIFIED_EXTERNALLY: + return "WorkflowPropertiesModifiedExternally" + case EVENT_TYPE_ACTIVITY_PROPERTIES_MODIFIED_EXTERNALLY: + return "ActivityPropertiesModifiedExternally" + case EVENT_TYPE_WORKFLOW_PROPERTIES_MODIFIED: + return "WorkflowPropertiesModified" + case EVENT_TYPE_NEXUS_OPERATION_SCHEDULED: + return "NexusOperationScheduled" + case EVENT_TYPE_NEXUS_OPERATION_STARTED: + return "NexusOperationStarted" + case EVENT_TYPE_NEXUS_OPERATION_COMPLETED: + return "NexusOperationCompleted" + case EVENT_TYPE_NEXUS_OPERATION_FAILED: + return "NexusOperationFailed" + case EVENT_TYPE_NEXUS_OPERATION_CANCELED: + return "NexusOperationCanceled" + case EVENT_TYPE_NEXUS_OPERATION_TIMED_OUT: + return "NexusOperationTimedOut" + case EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUESTED: + return "NexusOperationCancelRequested" + default: + return strconv.Itoa(int(x)) + } + +} + +func (EventType) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_event_type_proto_enumTypes[0].Descriptor() +} + +func (EventType) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_event_type_proto_enumTypes[0] +} + +func (x EventType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +func (EventType) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_event_type_proto_rawDescGZIP(), []int{0} +} + +var File_temporal_api_enums_v1_event_type_proto protoreflect.FileDescriptor + +var file_temporal_api_enums_v1_event_type_proto_rawDesc = []byte{ + 0x0a, 0x26, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, + 0x70, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2a, + 0xe8, 0x13, 0x0a, 0x09, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, + 0x16, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, + 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x29, 0x0a, 0x25, 0x45, 0x56, 0x45, + 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, + 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, + 0x45, 0x44, 0x10, 0x01, 0x12, 0x2b, 0x0a, 0x27, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, + 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, + 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, + 0x02, 0x12, 0x28, 0x0a, 0x24, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x12, 0x2b, 0x0a, 0x27, 0x45, + 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, + 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x49, 0x4d, + 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x04, 0x12, 0x26, 0x0a, 0x22, 0x45, 0x56, 0x45, 0x4e, + 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, + 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x44, 0x10, 0x05, + 0x12, 0x24, 0x0a, 0x20, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, + 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, + 0x52, 0x54, 0x45, 0x44, 0x10, 0x06, 0x12, 0x26, 0x0a, 0x22, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, + 0x53, 0x4b, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x07, 0x12, 0x26, + 0x0a, 0x22, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, + 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x44, + 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x08, 0x12, 0x23, 0x0a, 0x1f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, + 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x09, 0x12, 0x26, 0x0a, 0x22, 0x45, + 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x56, 0x49, + 0x54, 0x59, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, + 0x44, 0x10, 0x0a, 0x12, 0x24, 0x0a, 0x20, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, + 0x45, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, + 0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, 0x10, 0x0b, 0x12, 0x26, 0x0a, 0x22, 0x45, 0x56, 0x45, + 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, + 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, + 0x0c, 0x12, 0x23, 0x0a, 0x1f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, + 0x41, 0x43, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, + 0x49, 0x4c, 0x45, 0x44, 0x10, 0x0d, 0x12, 0x26, 0x0a, 0x22, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x54, 0x41, + 0x53, 0x4b, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x0e, 0x12, 0x2d, + 0x0a, 0x29, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x41, 0x43, 0x54, + 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, + 0x4c, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x45, 0x44, 0x10, 0x0f, 0x12, 0x25, 0x0a, + 0x21, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x41, 0x43, 0x54, 0x49, + 0x56, 0x49, 0x54, 0x59, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, + 0x45, 0x44, 0x10, 0x10, 0x12, 0x1c, 0x0a, 0x18, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, + 0x50, 0x45, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, + 0x10, 0x11, 0x12, 0x1a, 0x0a, 0x16, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, + 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x52, 0x5f, 0x46, 0x49, 0x52, 0x45, 0x44, 0x10, 0x12, 0x12, 0x1d, + 0x0a, 0x19, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x54, 0x49, 0x4d, + 0x45, 0x52, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x45, 0x44, 0x10, 0x13, 0x12, 0x32, 0x0a, + 0x2e, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, + 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, + 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x45, 0x44, 0x10, + 0x14, 0x12, 0x2a, 0x0a, 0x26, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x45, 0x44, 0x10, 0x15, 0x12, 0x43, 0x0a, + 0x3f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x52, 0x45, 0x51, 0x55, + 0x45, 0x53, 0x54, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, + 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, + 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41, 0x54, 0x45, 0x44, + 0x10, 0x16, 0x12, 0x40, 0x0a, 0x3c, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, + 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, + 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, + 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, 0x41, 0x49, 0x4c, + 0x45, 0x44, 0x10, 0x17, 0x12, 0x3b, 0x0a, 0x37, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, + 0x50, 0x45, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, + 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, + 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x45, 0x44, 0x10, + 0x18, 0x12, 0x1e, 0x0a, 0x1a, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, + 0x4d, 0x41, 0x52, 0x4b, 0x45, 0x52, 0x5f, 0x52, 0x45, 0x43, 0x4f, 0x52, 0x44, 0x45, 0x44, 0x10, + 0x19, 0x12, 0x2a, 0x0a, 0x26, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x53, 0x49, 0x47, 0x4e, 0x41, 0x4c, 0x45, 0x44, 0x10, 0x1a, 0x12, 0x2c, 0x0a, + 0x28, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, + 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, + 0x45, 0x52, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, 0x44, 0x10, 0x1b, 0x12, 0x32, 0x0a, 0x2e, 0x45, + 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, + 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x4f, 0x4e, + 0x54, 0x49, 0x4e, 0x55, 0x45, 0x44, 0x5f, 0x41, 0x53, 0x5f, 0x4e, 0x45, 0x57, 0x10, 0x1c, 0x12, + 0x37, 0x0a, 0x33, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, + 0x41, 0x52, 0x54, 0x5f, 0x43, 0x48, 0x49, 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, + 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x49, 0x4e, 0x49, + 0x54, 0x49, 0x41, 0x54, 0x45, 0x44, 0x10, 0x1d, 0x12, 0x34, 0x0a, 0x30, 0x45, 0x56, 0x45, 0x4e, + 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x43, 0x48, 0x49, + 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, + 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x1e, 0x12, 0x2f, + 0x0a, 0x2b, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x48, 0x49, + 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, + 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, 0x10, 0x1f, 0x12, + 0x31, 0x0a, 0x2d, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x48, + 0x49, 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, + 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, + 0x10, 0x20, 0x12, 0x2e, 0x0a, 0x2a, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, + 0x5f, 0x43, 0x48, 0x49, 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, + 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, + 0x10, 0x21, 0x12, 0x30, 0x0a, 0x2c, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, + 0x5f, 0x43, 0x48, 0x49, 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, + 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, + 0x45, 0x44, 0x10, 0x22, 0x12, 0x31, 0x0a, 0x2d, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, + 0x50, 0x45, 0x5f, 0x43, 0x48, 0x49, 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, + 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x49, 0x4d, 0x45, + 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x23, 0x12, 0x32, 0x0a, 0x2e, 0x45, 0x56, 0x45, 0x4e, 0x54, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x48, 0x49, 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, + 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, + 0x45, 0x52, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, 0x44, 0x10, 0x24, 0x12, 0x3b, 0x0a, 0x37, 0x45, + 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x49, 0x47, 0x4e, 0x41, 0x4c, + 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, + 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x49, 0x4e, 0x49, + 0x54, 0x49, 0x41, 0x54, 0x45, 0x44, 0x10, 0x25, 0x12, 0x38, 0x0a, 0x34, 0x45, 0x56, 0x45, 0x4e, + 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x49, 0x47, 0x4e, 0x41, 0x4c, 0x5f, 0x45, 0x58, + 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, + 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, + 0x10, 0x26, 0x12, 0x33, 0x0a, 0x2f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, + 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, + 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x49, 0x47, + 0x4e, 0x41, 0x4c, 0x45, 0x44, 0x10, 0x27, 0x12, 0x30, 0x0a, 0x2c, 0x45, 0x56, 0x45, 0x4e, 0x54, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x5f, 0x57, 0x4f, 0x52, + 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x53, 0x45, 0x41, 0x52, 0x43, 0x48, 0x5f, 0x41, 0x54, 0x54, + 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, 0x28, 0x12, 0x31, 0x0a, 0x2d, 0x45, 0x56, 0x45, + 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, + 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, + 0x45, 0x5f, 0x41, 0x44, 0x4d, 0x49, 0x54, 0x54, 0x45, 0x44, 0x10, 0x2f, 0x12, 0x31, 0x0a, 0x2d, + 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x50, + 0x44, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x43, 0x43, 0x45, 0x50, 0x54, 0x45, 0x44, 0x10, 0x29, 0x12, + 0x31, 0x0a, 0x2d, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, + 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, + 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, 0x52, 0x45, 0x4a, 0x45, 0x43, 0x54, 0x45, 0x44, + 0x10, 0x2a, 0x12, 0x32, 0x0a, 0x2e, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, + 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, + 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, + 0x45, 0x54, 0x45, 0x44, 0x10, 0x2b, 0x12, 0x36, 0x0a, 0x32, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x50, 0x52, + 0x4f, 0x50, 0x45, 0x52, 0x54, 0x49, 0x45, 0x53, 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, + 0x44, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x4c, 0x59, 0x10, 0x2c, 0x12, 0x36, + 0x0a, 0x32, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x41, 0x43, 0x54, + 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x50, 0x52, 0x4f, 0x50, 0x45, 0x52, 0x54, 0x49, 0x45, 0x53, + 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, 0x44, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, + 0x41, 0x4c, 0x4c, 0x59, 0x10, 0x2d, 0x12, 0x2b, 0x0a, 0x27, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x50, 0x52, + 0x4f, 0x50, 0x45, 0x52, 0x54, 0x49, 0x45, 0x53, 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, + 0x44, 0x10, 0x2e, 0x12, 0x28, 0x0a, 0x24, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, + 0x45, 0x5f, 0x4e, 0x45, 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, + 0x4e, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x44, 0x10, 0x30, 0x12, 0x26, 0x0a, + 0x22, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4e, 0x45, 0x58, 0x55, + 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x52, + 0x54, 0x45, 0x44, 0x10, 0x31, 0x12, 0x28, 0x0a, 0x24, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, + 0x59, 0x50, 0x45, 0x5f, 0x4e, 0x45, 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, + 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x32, 0x12, + 0x25, 0x0a, 0x21, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4e, 0x45, + 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, 0x41, + 0x49, 0x4c, 0x45, 0x44, 0x10, 0x33, 0x12, 0x27, 0x0a, 0x23, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4e, 0x45, 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, + 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x45, 0x44, 0x10, 0x34, 0x12, + 0x28, 0x0a, 0x24, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4e, 0x45, + 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x49, + 0x4d, 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x35, 0x12, 0x2f, 0x0a, 0x2b, 0x45, 0x56, 0x45, + 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4e, 0x45, 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, + 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x52, + 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x45, 0x44, 0x10, 0x36, 0x42, 0x86, 0x01, 0x0a, 0x18, 0x69, + 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x42, 0x0e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, + 0x70, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x21, 0x67, 0x6f, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, + 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x3b, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0xaa, 0x02, 0x17, 0x54, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x45, 0x6e, + 0x75, 0x6d, 0x73, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1a, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x3a, + 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_enums_v1_event_type_proto_rawDescOnce sync.Once + file_temporal_api_enums_v1_event_type_proto_rawDescData = file_temporal_api_enums_v1_event_type_proto_rawDesc +) + +func file_temporal_api_enums_v1_event_type_proto_rawDescGZIP() []byte { + file_temporal_api_enums_v1_event_type_proto_rawDescOnce.Do(func() { + file_temporal_api_enums_v1_event_type_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_enums_v1_event_type_proto_rawDescData) + }) + return file_temporal_api_enums_v1_event_type_proto_rawDescData +} + +var file_temporal_api_enums_v1_event_type_proto_enumTypes = make([]protoimpl.EnumInfo, 1) +var file_temporal_api_enums_v1_event_type_proto_goTypes = []any{ + (EventType)(0), // 0: temporal.api.enums.v1.EventType +} +var file_temporal_api_enums_v1_event_type_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_enums_v1_event_type_proto_init() } +func file_temporal_api_enums_v1_event_type_proto_init() { + if File_temporal_api_enums_v1_event_type_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_enums_v1_event_type_proto_rawDesc, + NumEnums: 1, + NumMessages: 0, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_enums_v1_event_type_proto_goTypes, + DependencyIndexes: file_temporal_api_enums_v1_event_type_proto_depIdxs, + EnumInfos: file_temporal_api_enums_v1_event_type_proto_enumTypes, + }.Build() + File_temporal_api_enums_v1_event_type_proto = out.File + file_temporal_api_enums_v1_event_type_proto_rawDesc = nil + file_temporal_api_enums_v1_event_type_proto_goTypes = nil + file_temporal_api_enums_v1_event_type_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/enums/v1/failed_cause.go-helpers.pb.go b/vendor/go.temporal.io/api/enums/v1/failed_cause.go-helpers.pb.go new file mode 100644 index 00000000000..2f2e7b11c3c --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/failed_cause.go-helpers.pb.go @@ -0,0 +1,182 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package enums + +import ( + "fmt" +) + +var ( + WorkflowTaskFailedCause_shorthandValue = map[string]int32{ + "Unspecified": 0, + "UnhandledCommand": 1, + "BadScheduleActivityAttributes": 2, + "BadRequestCancelActivityAttributes": 3, + "BadStartTimerAttributes": 4, + "BadCancelTimerAttributes": 5, + "BadRecordMarkerAttributes": 6, + "BadCompleteWorkflowExecutionAttributes": 7, + "BadFailWorkflowExecutionAttributes": 8, + "BadCancelWorkflowExecutionAttributes": 9, + "BadRequestCancelExternalWorkflowExecutionAttributes": 10, + "BadContinueAsNewAttributes": 11, + "StartTimerDuplicateId": 12, + "ResetStickyTaskQueue": 13, + "WorkflowWorkerUnhandledFailure": 14, + "BadSignalWorkflowExecutionAttributes": 15, + "BadStartChildExecutionAttributes": 16, + "ForceCloseCommand": 17, + "FailoverCloseCommand": 18, + "BadSignalInputSize": 19, + "ResetWorkflow": 20, + "BadBinary": 21, + "ScheduleActivityDuplicateId": 22, + "BadSearchAttributes": 23, + "NonDeterministicError": 24, + "BadModifyWorkflowPropertiesAttributes": 25, + "PendingChildWorkflowsLimitExceeded": 26, + "PendingActivitiesLimitExceeded": 27, + "PendingSignalsLimitExceeded": 28, + "PendingRequestCancelLimitExceeded": 29, + "BadUpdateWorkflowExecutionMessage": 30, + "UnhandledUpdate": 31, + "BadScheduleNexusOperationAttributes": 32, + "PendingNexusOperationsLimitExceeded": 33, + "BadRequestCancelNexusOperationAttributes": 34, + "FeatureDisabled": 35, + } +) + +// WorkflowTaskFailedCauseFromString parses a WorkflowTaskFailedCause value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to WorkflowTaskFailedCause +func WorkflowTaskFailedCauseFromString(s string) (WorkflowTaskFailedCause, error) { + if v, ok := WorkflowTaskFailedCause_value[s]; ok { + return WorkflowTaskFailedCause(v), nil + } else if v, ok := WorkflowTaskFailedCause_shorthandValue[s]; ok { + return WorkflowTaskFailedCause(v), nil + } + return WorkflowTaskFailedCause(0), fmt.Errorf("%s is not a valid WorkflowTaskFailedCause", s) +} + +var ( + StartChildWorkflowExecutionFailedCause_shorthandValue = map[string]int32{ + "Unspecified": 0, + "WorkflowAlreadyExists": 1, + "NamespaceNotFound": 2, + } +) + +// StartChildWorkflowExecutionFailedCauseFromString parses a StartChildWorkflowExecutionFailedCause value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to StartChildWorkflowExecutionFailedCause +func StartChildWorkflowExecutionFailedCauseFromString(s string) (StartChildWorkflowExecutionFailedCause, error) { + if v, ok := StartChildWorkflowExecutionFailedCause_value[s]; ok { + return StartChildWorkflowExecutionFailedCause(v), nil + } else if v, ok := StartChildWorkflowExecutionFailedCause_shorthandValue[s]; ok { + return StartChildWorkflowExecutionFailedCause(v), nil + } + return StartChildWorkflowExecutionFailedCause(0), fmt.Errorf("%s is not a valid StartChildWorkflowExecutionFailedCause", s) +} + +var ( + CancelExternalWorkflowExecutionFailedCause_shorthandValue = map[string]int32{ + "Unspecified": 0, + "ExternalWorkflowExecutionNotFound": 1, + "NamespaceNotFound": 2, + } +) + +// CancelExternalWorkflowExecutionFailedCauseFromString parses a CancelExternalWorkflowExecutionFailedCause value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to CancelExternalWorkflowExecutionFailedCause +func CancelExternalWorkflowExecutionFailedCauseFromString(s string) (CancelExternalWorkflowExecutionFailedCause, error) { + if v, ok := CancelExternalWorkflowExecutionFailedCause_value[s]; ok { + return CancelExternalWorkflowExecutionFailedCause(v), nil + } else if v, ok := CancelExternalWorkflowExecutionFailedCause_shorthandValue[s]; ok { + return CancelExternalWorkflowExecutionFailedCause(v), nil + } + return CancelExternalWorkflowExecutionFailedCause(0), fmt.Errorf("%s is not a valid CancelExternalWorkflowExecutionFailedCause", s) +} + +var ( + SignalExternalWorkflowExecutionFailedCause_shorthandValue = map[string]int32{ + "Unspecified": 0, + "ExternalWorkflowExecutionNotFound": 1, + "NamespaceNotFound": 2, + "SignalCountLimitExceeded": 3, + } +) + +// SignalExternalWorkflowExecutionFailedCauseFromString parses a SignalExternalWorkflowExecutionFailedCause value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to SignalExternalWorkflowExecutionFailedCause +func SignalExternalWorkflowExecutionFailedCauseFromString(s string) (SignalExternalWorkflowExecutionFailedCause, error) { + if v, ok := SignalExternalWorkflowExecutionFailedCause_value[s]; ok { + return SignalExternalWorkflowExecutionFailedCause(v), nil + } else if v, ok := SignalExternalWorkflowExecutionFailedCause_shorthandValue[s]; ok { + return SignalExternalWorkflowExecutionFailedCause(v), nil + } + return SignalExternalWorkflowExecutionFailedCause(0), fmt.Errorf("%s is not a valid SignalExternalWorkflowExecutionFailedCause", s) +} + +var ( + ResourceExhaustedCause_shorthandValue = map[string]int32{ + "Unspecified": 0, + "RpsLimit": 1, + "ConcurrentLimit": 2, + "SystemOverloaded": 3, + "PersistenceLimit": 4, + "BusyWorkflow": 5, + "ApsLimit": 6, + "PersistenceStorageLimit": 7, + "CircuitBreakerOpen": 8, + } +) + +// ResourceExhaustedCauseFromString parses a ResourceExhaustedCause value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to ResourceExhaustedCause +func ResourceExhaustedCauseFromString(s string) (ResourceExhaustedCause, error) { + if v, ok := ResourceExhaustedCause_value[s]; ok { + return ResourceExhaustedCause(v), nil + } else if v, ok := ResourceExhaustedCause_shorthandValue[s]; ok { + return ResourceExhaustedCause(v), nil + } + return ResourceExhaustedCause(0), fmt.Errorf("%s is not a valid ResourceExhaustedCause", s) +} + +var ( + ResourceExhaustedScope_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Namespace": 1, + "System": 2, + } +) + +// ResourceExhaustedScopeFromString parses a ResourceExhaustedScope value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to ResourceExhaustedScope +func ResourceExhaustedScopeFromString(s string) (ResourceExhaustedScope, error) { + if v, ok := ResourceExhaustedScope_value[s]; ok { + return ResourceExhaustedScope(v), nil + } else if v, ok := ResourceExhaustedScope_shorthandValue[s]; ok { + return ResourceExhaustedScope(v), nil + } + return ResourceExhaustedScope(0), fmt.Errorf("%s is not a valid ResourceExhaustedScope", s) +} diff --git a/vendor/go.temporal.io/api/enums/v1/failed_cause.pb.go b/vendor/go.temporal.io/api/enums/v1/failed_cause.pb.go new file mode 100644 index 00000000000..688729dcdb9 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/failed_cause.pb.go @@ -0,0 +1,954 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/enums/v1/failed_cause.proto + +package enums + +import ( + reflect "reflect" + "strconv" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// Workflow tasks can fail for various reasons. Note that some of these reasons can only originate +// from the server, and some of them can only originate from the SDK/worker. +type WorkflowTaskFailedCause int32 + +const ( + WORKFLOW_TASK_FAILED_CAUSE_UNSPECIFIED WorkflowTaskFailedCause = 0 + // Between starting and completing the workflow task (with a workflow completion command), some + // new command (like a signal) was processed into workflow history. The outstanding task will be + // failed with this reason, and a worker must pick up a new task. + WORKFLOW_TASK_FAILED_CAUSE_UNHANDLED_COMMAND WorkflowTaskFailedCause = 1 + WORKFLOW_TASK_FAILED_CAUSE_BAD_SCHEDULE_ACTIVITY_ATTRIBUTES WorkflowTaskFailedCause = 2 + WORKFLOW_TASK_FAILED_CAUSE_BAD_REQUEST_CANCEL_ACTIVITY_ATTRIBUTES WorkflowTaskFailedCause = 3 + WORKFLOW_TASK_FAILED_CAUSE_BAD_START_TIMER_ATTRIBUTES WorkflowTaskFailedCause = 4 + WORKFLOW_TASK_FAILED_CAUSE_BAD_CANCEL_TIMER_ATTRIBUTES WorkflowTaskFailedCause = 5 + WORKFLOW_TASK_FAILED_CAUSE_BAD_RECORD_MARKER_ATTRIBUTES WorkflowTaskFailedCause = 6 + WORKFLOW_TASK_FAILED_CAUSE_BAD_COMPLETE_WORKFLOW_EXECUTION_ATTRIBUTES WorkflowTaskFailedCause = 7 + WORKFLOW_TASK_FAILED_CAUSE_BAD_FAIL_WORKFLOW_EXECUTION_ATTRIBUTES WorkflowTaskFailedCause = 8 + WORKFLOW_TASK_FAILED_CAUSE_BAD_CANCEL_WORKFLOW_EXECUTION_ATTRIBUTES WorkflowTaskFailedCause = 9 + WORKFLOW_TASK_FAILED_CAUSE_BAD_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_ATTRIBUTES WorkflowTaskFailedCause = 10 + WORKFLOW_TASK_FAILED_CAUSE_BAD_CONTINUE_AS_NEW_ATTRIBUTES WorkflowTaskFailedCause = 11 + WORKFLOW_TASK_FAILED_CAUSE_START_TIMER_DUPLICATE_ID WorkflowTaskFailedCause = 12 + // The worker wishes to fail the task and have the next one be generated on a normal, not sticky + // queue. Generally workers should prefer to use the explicit `ResetStickyTaskQueue` RPC call. + WORKFLOW_TASK_FAILED_CAUSE_RESET_STICKY_TASK_QUEUE WorkflowTaskFailedCause = 13 + WORKFLOW_TASK_FAILED_CAUSE_WORKFLOW_WORKER_UNHANDLED_FAILURE WorkflowTaskFailedCause = 14 + WORKFLOW_TASK_FAILED_CAUSE_BAD_SIGNAL_WORKFLOW_EXECUTION_ATTRIBUTES WorkflowTaskFailedCause = 15 + WORKFLOW_TASK_FAILED_CAUSE_BAD_START_CHILD_EXECUTION_ATTRIBUTES WorkflowTaskFailedCause = 16 + WORKFLOW_TASK_FAILED_CAUSE_FORCE_CLOSE_COMMAND WorkflowTaskFailedCause = 17 + WORKFLOW_TASK_FAILED_CAUSE_FAILOVER_CLOSE_COMMAND WorkflowTaskFailedCause = 18 + WORKFLOW_TASK_FAILED_CAUSE_BAD_SIGNAL_INPUT_SIZE WorkflowTaskFailedCause = 19 + WORKFLOW_TASK_FAILED_CAUSE_RESET_WORKFLOW WorkflowTaskFailedCause = 20 + WORKFLOW_TASK_FAILED_CAUSE_BAD_BINARY WorkflowTaskFailedCause = 21 + WORKFLOW_TASK_FAILED_CAUSE_SCHEDULE_ACTIVITY_DUPLICATE_ID WorkflowTaskFailedCause = 22 + WORKFLOW_TASK_FAILED_CAUSE_BAD_SEARCH_ATTRIBUTES WorkflowTaskFailedCause = 23 + // The worker encountered a mismatch while replaying history between what was expected, and + // what the workflow code actually did. + WORKFLOW_TASK_FAILED_CAUSE_NON_DETERMINISTIC_ERROR WorkflowTaskFailedCause = 24 + WORKFLOW_TASK_FAILED_CAUSE_BAD_MODIFY_WORKFLOW_PROPERTIES_ATTRIBUTES WorkflowTaskFailedCause = 25 + // We send the below error codes to users when their requests would violate a size constraint + // of their workflow. We do this to ensure that the state of their workflow does not become too + // large because that can cause severe performance degradation. You can modify the thresholds for + // each of these errors within your dynamic config. + // + // Spawning a new child workflow would cause this workflow to exceed its limit of pending child + // workflows. + WORKFLOW_TASK_FAILED_CAUSE_PENDING_CHILD_WORKFLOWS_LIMIT_EXCEEDED WorkflowTaskFailedCause = 26 + // Starting a new activity would cause this workflow to exceed its limit of pending activities + // that we track. + WORKFLOW_TASK_FAILED_CAUSE_PENDING_ACTIVITIES_LIMIT_EXCEEDED WorkflowTaskFailedCause = 27 + // A workflow has a buffer of signals that have not yet reached their destination. We return this + // error when sending a new signal would exceed the capacity of this buffer. + WORKFLOW_TASK_FAILED_CAUSE_PENDING_SIGNALS_LIMIT_EXCEEDED WorkflowTaskFailedCause = 28 + // Similarly, we have a buffer of pending requests to cancel other workflows. We return this error + // when our capacity for pending cancel requests is already reached. + WORKFLOW_TASK_FAILED_CAUSE_PENDING_REQUEST_CANCEL_LIMIT_EXCEEDED WorkflowTaskFailedCause = 29 + // Workflow execution update message (update.Acceptance, update.Rejection, or update.Response) + // has wrong format, or missing required fields. + WORKFLOW_TASK_FAILED_CAUSE_BAD_UPDATE_WORKFLOW_EXECUTION_MESSAGE WorkflowTaskFailedCause = 30 + // Similar to WORKFLOW_TASK_FAILED_CAUSE_UNHANDLED_COMMAND, but for updates. + WORKFLOW_TASK_FAILED_CAUSE_UNHANDLED_UPDATE WorkflowTaskFailedCause = 31 + // A workflow task completed with an invalid ScheduleNexusOperation command. + WORKFLOW_TASK_FAILED_CAUSE_BAD_SCHEDULE_NEXUS_OPERATION_ATTRIBUTES WorkflowTaskFailedCause = 32 + // A workflow task completed requesting to schedule a Nexus Operation exceeding the server configured limit. + WORKFLOW_TASK_FAILED_CAUSE_PENDING_NEXUS_OPERATIONS_LIMIT_EXCEEDED WorkflowTaskFailedCause = 33 + // A workflow task completed with an invalid RequestCancelNexusOperation command. + WORKFLOW_TASK_FAILED_CAUSE_BAD_REQUEST_CANCEL_NEXUS_OPERATION_ATTRIBUTES WorkflowTaskFailedCause = 34 + // A workflow task completed requesting a feature that's disabled on the server (either system wide or - typically - + // for the workflow's namespace). + // Check the workflow task failure message for more information. + WORKFLOW_TASK_FAILED_CAUSE_FEATURE_DISABLED WorkflowTaskFailedCause = 35 +) + +// Enum value maps for WorkflowTaskFailedCause. +var ( + WorkflowTaskFailedCause_name = map[int32]string{ + 0: "WORKFLOW_TASK_FAILED_CAUSE_UNSPECIFIED", + 1: "WORKFLOW_TASK_FAILED_CAUSE_UNHANDLED_COMMAND", + 2: "WORKFLOW_TASK_FAILED_CAUSE_BAD_SCHEDULE_ACTIVITY_ATTRIBUTES", + 3: "WORKFLOW_TASK_FAILED_CAUSE_BAD_REQUEST_CANCEL_ACTIVITY_ATTRIBUTES", + 4: "WORKFLOW_TASK_FAILED_CAUSE_BAD_START_TIMER_ATTRIBUTES", + 5: "WORKFLOW_TASK_FAILED_CAUSE_BAD_CANCEL_TIMER_ATTRIBUTES", + 6: "WORKFLOW_TASK_FAILED_CAUSE_BAD_RECORD_MARKER_ATTRIBUTES", + 7: "WORKFLOW_TASK_FAILED_CAUSE_BAD_COMPLETE_WORKFLOW_EXECUTION_ATTRIBUTES", + 8: "WORKFLOW_TASK_FAILED_CAUSE_BAD_FAIL_WORKFLOW_EXECUTION_ATTRIBUTES", + 9: "WORKFLOW_TASK_FAILED_CAUSE_BAD_CANCEL_WORKFLOW_EXECUTION_ATTRIBUTES", + 10: "WORKFLOW_TASK_FAILED_CAUSE_BAD_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_ATTRIBUTES", + 11: "WORKFLOW_TASK_FAILED_CAUSE_BAD_CONTINUE_AS_NEW_ATTRIBUTES", + 12: "WORKFLOW_TASK_FAILED_CAUSE_START_TIMER_DUPLICATE_ID", + 13: "WORKFLOW_TASK_FAILED_CAUSE_RESET_STICKY_TASK_QUEUE", + 14: "WORKFLOW_TASK_FAILED_CAUSE_WORKFLOW_WORKER_UNHANDLED_FAILURE", + 15: "WORKFLOW_TASK_FAILED_CAUSE_BAD_SIGNAL_WORKFLOW_EXECUTION_ATTRIBUTES", + 16: "WORKFLOW_TASK_FAILED_CAUSE_BAD_START_CHILD_EXECUTION_ATTRIBUTES", + 17: "WORKFLOW_TASK_FAILED_CAUSE_FORCE_CLOSE_COMMAND", + 18: "WORKFLOW_TASK_FAILED_CAUSE_FAILOVER_CLOSE_COMMAND", + 19: "WORKFLOW_TASK_FAILED_CAUSE_BAD_SIGNAL_INPUT_SIZE", + 20: "WORKFLOW_TASK_FAILED_CAUSE_RESET_WORKFLOW", + 21: "WORKFLOW_TASK_FAILED_CAUSE_BAD_BINARY", + 22: "WORKFLOW_TASK_FAILED_CAUSE_SCHEDULE_ACTIVITY_DUPLICATE_ID", + 23: "WORKFLOW_TASK_FAILED_CAUSE_BAD_SEARCH_ATTRIBUTES", + 24: "WORKFLOW_TASK_FAILED_CAUSE_NON_DETERMINISTIC_ERROR", + 25: "WORKFLOW_TASK_FAILED_CAUSE_BAD_MODIFY_WORKFLOW_PROPERTIES_ATTRIBUTES", + 26: "WORKFLOW_TASK_FAILED_CAUSE_PENDING_CHILD_WORKFLOWS_LIMIT_EXCEEDED", + 27: "WORKFLOW_TASK_FAILED_CAUSE_PENDING_ACTIVITIES_LIMIT_EXCEEDED", + 28: "WORKFLOW_TASK_FAILED_CAUSE_PENDING_SIGNALS_LIMIT_EXCEEDED", + 29: "WORKFLOW_TASK_FAILED_CAUSE_PENDING_REQUEST_CANCEL_LIMIT_EXCEEDED", + 30: "WORKFLOW_TASK_FAILED_CAUSE_BAD_UPDATE_WORKFLOW_EXECUTION_MESSAGE", + 31: "WORKFLOW_TASK_FAILED_CAUSE_UNHANDLED_UPDATE", + 32: "WORKFLOW_TASK_FAILED_CAUSE_BAD_SCHEDULE_NEXUS_OPERATION_ATTRIBUTES", + 33: "WORKFLOW_TASK_FAILED_CAUSE_PENDING_NEXUS_OPERATIONS_LIMIT_EXCEEDED", + 34: "WORKFLOW_TASK_FAILED_CAUSE_BAD_REQUEST_CANCEL_NEXUS_OPERATION_ATTRIBUTES", + 35: "WORKFLOW_TASK_FAILED_CAUSE_FEATURE_DISABLED", + } + WorkflowTaskFailedCause_value = map[string]int32{ + "WORKFLOW_TASK_FAILED_CAUSE_UNSPECIFIED": 0, + "WORKFLOW_TASK_FAILED_CAUSE_UNHANDLED_COMMAND": 1, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_SCHEDULE_ACTIVITY_ATTRIBUTES": 2, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_REQUEST_CANCEL_ACTIVITY_ATTRIBUTES": 3, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_START_TIMER_ATTRIBUTES": 4, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_CANCEL_TIMER_ATTRIBUTES": 5, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_RECORD_MARKER_ATTRIBUTES": 6, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_COMPLETE_WORKFLOW_EXECUTION_ATTRIBUTES": 7, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_FAIL_WORKFLOW_EXECUTION_ATTRIBUTES": 8, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_CANCEL_WORKFLOW_EXECUTION_ATTRIBUTES": 9, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_ATTRIBUTES": 10, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_CONTINUE_AS_NEW_ATTRIBUTES": 11, + "WORKFLOW_TASK_FAILED_CAUSE_START_TIMER_DUPLICATE_ID": 12, + "WORKFLOW_TASK_FAILED_CAUSE_RESET_STICKY_TASK_QUEUE": 13, + "WORKFLOW_TASK_FAILED_CAUSE_WORKFLOW_WORKER_UNHANDLED_FAILURE": 14, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_SIGNAL_WORKFLOW_EXECUTION_ATTRIBUTES": 15, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_START_CHILD_EXECUTION_ATTRIBUTES": 16, + "WORKFLOW_TASK_FAILED_CAUSE_FORCE_CLOSE_COMMAND": 17, + "WORKFLOW_TASK_FAILED_CAUSE_FAILOVER_CLOSE_COMMAND": 18, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_SIGNAL_INPUT_SIZE": 19, + "WORKFLOW_TASK_FAILED_CAUSE_RESET_WORKFLOW": 20, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_BINARY": 21, + "WORKFLOW_TASK_FAILED_CAUSE_SCHEDULE_ACTIVITY_DUPLICATE_ID": 22, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_SEARCH_ATTRIBUTES": 23, + "WORKFLOW_TASK_FAILED_CAUSE_NON_DETERMINISTIC_ERROR": 24, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_MODIFY_WORKFLOW_PROPERTIES_ATTRIBUTES": 25, + "WORKFLOW_TASK_FAILED_CAUSE_PENDING_CHILD_WORKFLOWS_LIMIT_EXCEEDED": 26, + "WORKFLOW_TASK_FAILED_CAUSE_PENDING_ACTIVITIES_LIMIT_EXCEEDED": 27, + "WORKFLOW_TASK_FAILED_CAUSE_PENDING_SIGNALS_LIMIT_EXCEEDED": 28, + "WORKFLOW_TASK_FAILED_CAUSE_PENDING_REQUEST_CANCEL_LIMIT_EXCEEDED": 29, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_UPDATE_WORKFLOW_EXECUTION_MESSAGE": 30, + "WORKFLOW_TASK_FAILED_CAUSE_UNHANDLED_UPDATE": 31, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_SCHEDULE_NEXUS_OPERATION_ATTRIBUTES": 32, + "WORKFLOW_TASK_FAILED_CAUSE_PENDING_NEXUS_OPERATIONS_LIMIT_EXCEEDED": 33, + "WORKFLOW_TASK_FAILED_CAUSE_BAD_REQUEST_CANCEL_NEXUS_OPERATION_ATTRIBUTES": 34, + "WORKFLOW_TASK_FAILED_CAUSE_FEATURE_DISABLED": 35, + } +) + +func (x WorkflowTaskFailedCause) Enum() *WorkflowTaskFailedCause { + p := new(WorkflowTaskFailedCause) + *p = x + return p +} + +func (x WorkflowTaskFailedCause) String() string { + switch x { + case WORKFLOW_TASK_FAILED_CAUSE_UNSPECIFIED: + return "Unspecified" + case WORKFLOW_TASK_FAILED_CAUSE_UNHANDLED_COMMAND: + return "UnhandledCommand" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_SCHEDULE_ACTIVITY_ATTRIBUTES: + return "BadScheduleActivityAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_REQUEST_CANCEL_ACTIVITY_ATTRIBUTES: + return "BadRequestCancelActivityAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_START_TIMER_ATTRIBUTES: + return "BadStartTimerAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_CANCEL_TIMER_ATTRIBUTES: + + // Deprecated: Use WorkflowTaskFailedCause.Descriptor instead. + return "BadCancelTimerAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_RECORD_MARKER_ATTRIBUTES: + return "BadRecordMarkerAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_COMPLETE_WORKFLOW_EXECUTION_ATTRIBUTES: + return "BadCompleteWorkflowExecutionAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_FAIL_WORKFLOW_EXECUTION_ATTRIBUTES: + return "BadFailWorkflowExecutionAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_CANCEL_WORKFLOW_EXECUTION_ATTRIBUTES: + return "BadCancelWorkflowExecutionAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_ATTRIBUTES: + return "BadRequestCancelExternalWorkflowExecutionAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_CONTINUE_AS_NEW_ATTRIBUTES: + return "BadContinueAsNewAttributes" + + // Enum value maps for StartChildWorkflowExecutionFailedCause. + case WORKFLOW_TASK_FAILED_CAUSE_START_TIMER_DUPLICATE_ID: + return "StartTimerDuplicateId" + case WORKFLOW_TASK_FAILED_CAUSE_RESET_STICKY_TASK_QUEUE: + return "ResetStickyTaskQueue" + case WORKFLOW_TASK_FAILED_CAUSE_WORKFLOW_WORKER_UNHANDLED_FAILURE: + return "WorkflowWorkerUnhandledFailure" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_SIGNAL_WORKFLOW_EXECUTION_ATTRIBUTES: + return "BadSignalWorkflowExecutionAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_START_CHILD_EXECUTION_ATTRIBUTES: + return "BadStartChildExecutionAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_FORCE_CLOSE_COMMAND: + return "ForceCloseCommand" + case WORKFLOW_TASK_FAILED_CAUSE_FAILOVER_CLOSE_COMMAND: + return "FailoverCloseCommand" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_SIGNAL_INPUT_SIZE: + return "BadSignalInputSize" + case WORKFLOW_TASK_FAILED_CAUSE_RESET_WORKFLOW: + return "ResetWorkflow" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_BINARY: + return "BadBinary" + case WORKFLOW_TASK_FAILED_CAUSE_SCHEDULE_ACTIVITY_DUPLICATE_ID: + return "ScheduleActivityDuplicateId" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_SEARCH_ATTRIBUTES: + return "BadSearchAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_NON_DETERMINISTIC_ERROR: + return "NonDeterministicError" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_MODIFY_WORKFLOW_PROPERTIES_ATTRIBUTES: + return "BadModifyWorkflowPropertiesAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_PENDING_CHILD_WORKFLOWS_LIMIT_EXCEEDED: + return "PendingChildWorkflowsLimitExceeded" + case WORKFLOW_TASK_FAILED_CAUSE_PENDING_ACTIVITIES_LIMIT_EXCEEDED: + + // Deprecated: Use StartChildWorkflowExecutionFailedCause.Descriptor instead. + return "PendingActivitiesLimitExceeded" + case WORKFLOW_TASK_FAILED_CAUSE_PENDING_SIGNALS_LIMIT_EXCEEDED: + return "PendingSignalsLimitExceeded" + case WORKFLOW_TASK_FAILED_CAUSE_PENDING_REQUEST_CANCEL_LIMIT_EXCEEDED: + return "PendingRequestCancelLimitExceeded" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_UPDATE_WORKFLOW_EXECUTION_MESSAGE: + return "BadUpdateWorkflowExecutionMessage" + case WORKFLOW_TASK_FAILED_CAUSE_UNHANDLED_UPDATE: + return "UnhandledUpdate" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_SCHEDULE_NEXUS_OPERATION_ATTRIBUTES: + return "BadScheduleNexusOperationAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_PENDING_NEXUS_OPERATIONS_LIMIT_EXCEEDED: + return "PendingNexusOperationsLimitExceeded" + case WORKFLOW_TASK_FAILED_CAUSE_BAD_REQUEST_CANCEL_NEXUS_OPERATION_ATTRIBUTES: + return "BadRequestCancelNexusOperationAttributes" + case WORKFLOW_TASK_FAILED_CAUSE_FEATURE_DISABLED: + + // Enum value maps for CancelExternalWorkflowExecutionFailedCause. + return "FeatureDisabled" + default: + return strconv.Itoa(int(x)) + } + +} + +func (WorkflowTaskFailedCause) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_failed_cause_proto_enumTypes[0].Descriptor() +} + +func (WorkflowTaskFailedCause) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_failed_cause_proto_enumTypes[0] +} + +func (x WorkflowTaskFailedCause) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +func (WorkflowTaskFailedCause) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_failed_cause_proto_rawDescGZIP(), []int{0} +} + +type StartChildWorkflowExecutionFailedCause int32 + +const ( + START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_UNSPECIFIED StartChildWorkflowExecutionFailedCause = 0 + START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_WORKFLOW_ALREADY_EXISTS StartChildWorkflowExecutionFailedCause = 1 + START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND StartChildWorkflowExecutionFailedCause = 2 +) + +var ( + StartChildWorkflowExecutionFailedCause_name = map[int32]string{ + 0: "START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_UNSPECIFIED", + 1: "START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_WORKFLOW_ALREADY_EXISTS", + 2: "START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND", + } + StartChildWorkflowExecutionFailedCause_value = map[string]int32{ + "START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_UNSPECIFIED": 0, + "START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_WORKFLOW_ALREADY_EXISTS": 1, + "START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND": 2, + } +) + +func (x StartChildWorkflowExecutionFailedCause) Enum() *StartChildWorkflowExecutionFailedCause { + p := new(StartChildWorkflowExecutionFailedCause) + *p = x + return p +} + +func (x StartChildWorkflowExecutionFailedCause) String() string { + switch x { + case START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_UNSPECIFIED: + return "Unspecified" + case START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_WORKFLOW_ALREADY_EXISTS: + return "WorkflowAlreadyExists" + case START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND: + return "NamespaceNotFound" + default: + return strconv.Itoa(int(x)) + } + +} + +func (StartChildWorkflowExecutionFailedCause) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_failed_cause_proto_enumTypes[1].Descriptor() +} + +func (StartChildWorkflowExecutionFailedCause) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_failed_cause_proto_enumTypes[1] +} + +func (x StartChildWorkflowExecutionFailedCause) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +func (StartChildWorkflowExecutionFailedCause) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_failed_cause_proto_rawDescGZIP(), []int{1} +} + +type CancelExternalWorkflowExecutionFailedCause int32 + +const ( + CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_UNSPECIFIED CancelExternalWorkflowExecutionFailedCause = 0 + CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_EXTERNAL_WORKFLOW_EXECUTION_NOT_FOUND CancelExternalWorkflowExecutionFailedCause = 1 + CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND CancelExternalWorkflowExecutionFailedCause = 2 +) + +var ( + CancelExternalWorkflowExecutionFailedCause_name = map[int32]string{ + 0: "CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_UNSPECIFIED", + 1: "CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_EXTERNAL_WORKFLOW_EXECUTION_NOT_FOUND", + 2: "CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND", + } + CancelExternalWorkflowExecutionFailedCause_value = map[string]int32{ + "CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_UNSPECIFIED": 0, + "CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_EXTERNAL_WORKFLOW_EXECUTION_NOT_FOUND": 1, + "CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND": 2, + } +) + +func (x CancelExternalWorkflowExecutionFailedCause) Enum() *CancelExternalWorkflowExecutionFailedCause { + p := new(CancelExternalWorkflowExecutionFailedCause) + *p = x + return p +} + +func (x CancelExternalWorkflowExecutionFailedCause) String() string { + switch x { + case CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_UNSPECIFIED: + return "Unspecified" + case CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_EXTERNAL_WORKFLOW_EXECUTION_NOT_FOUND: + return "ExternalWorkflowExecutionNotFound" + case CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND: + return "NamespaceNotFound" + default: + return strconv.Itoa(int(x)) + } + +} + +func (CancelExternalWorkflowExecutionFailedCause) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_failed_cause_proto_enumTypes[2].Descriptor() +} + +func (CancelExternalWorkflowExecutionFailedCause) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_failed_cause_proto_enumTypes[2] +} + +func (x CancelExternalWorkflowExecutionFailedCause) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use CancelExternalWorkflowExecutionFailedCause.Descriptor instead. +func (CancelExternalWorkflowExecutionFailedCause) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_failed_cause_proto_rawDescGZIP(), []int{2} +} + +type SignalExternalWorkflowExecutionFailedCause int32 + +const ( + SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_UNSPECIFIED SignalExternalWorkflowExecutionFailedCause = 0 + SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_EXTERNAL_WORKFLOW_EXECUTION_NOT_FOUND SignalExternalWorkflowExecutionFailedCause = 1 + SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND SignalExternalWorkflowExecutionFailedCause = 2 + // Signal count limit is per workflow and controlled by server dynamic config "history.maximumSignalsPerExecution" + SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_SIGNAL_COUNT_LIMIT_EXCEEDED SignalExternalWorkflowExecutionFailedCause = 3 +) + +// Enum value maps for SignalExternalWorkflowExecutionFailedCause. +var ( + SignalExternalWorkflowExecutionFailedCause_name = map[int32]string{ + 0: "SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_UNSPECIFIED", + 1: "SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_EXTERNAL_WORKFLOW_EXECUTION_NOT_FOUND", + 2: "SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND", + 3: "SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_SIGNAL_COUNT_LIMIT_EXCEEDED", + } + SignalExternalWorkflowExecutionFailedCause_value = map[string]int32{ + "SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_UNSPECIFIED": 0, + "SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_EXTERNAL_WORKFLOW_EXECUTION_NOT_FOUND": 1, + "SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND": 2, + "SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_SIGNAL_COUNT_LIMIT_EXCEEDED": 3, + } +) + +func (x SignalExternalWorkflowExecutionFailedCause) Enum() *SignalExternalWorkflowExecutionFailedCause { + p := new(SignalExternalWorkflowExecutionFailedCause) + *p = x + return p +} + +func (x SignalExternalWorkflowExecutionFailedCause) String() string { + switch x { + case SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_UNSPECIFIED: + return "Unspecified" + case SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_EXTERNAL_WORKFLOW_EXECUTION_NOT_FOUND: + return "ExternalWorkflowExecutionNotFound" + case SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND: + return "NamespaceNotFound" + case SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_SIGNAL_COUNT_LIMIT_EXCEEDED: + return "SignalCountLimitExceeded" + default: + return strconv.Itoa(int(x)) + } + +} + +func (SignalExternalWorkflowExecutionFailedCause) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_failed_cause_proto_enumTypes[3].Descriptor() +} + +func (SignalExternalWorkflowExecutionFailedCause) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_failed_cause_proto_enumTypes[3] +} + +func (x SignalExternalWorkflowExecutionFailedCause) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SignalExternalWorkflowExecutionFailedCause.Descriptor instead. +func (SignalExternalWorkflowExecutionFailedCause) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_failed_cause_proto_rawDescGZIP(), []int{3} +} + +type ResourceExhaustedCause int32 + +const ( + RESOURCE_EXHAUSTED_CAUSE_UNSPECIFIED ResourceExhaustedCause = 0 + // Caller exceeds request per second limit. + RESOURCE_EXHAUSTED_CAUSE_RPS_LIMIT ResourceExhaustedCause = 1 + // Caller exceeds max concurrent request limit. + RESOURCE_EXHAUSTED_CAUSE_CONCURRENT_LIMIT ResourceExhaustedCause = 2 + // System overloaded. + RESOURCE_EXHAUSTED_CAUSE_SYSTEM_OVERLOADED ResourceExhaustedCause = 3 + // Namespace exceeds persistence rate limit. + RESOURCE_EXHAUSTED_CAUSE_PERSISTENCE_LIMIT ResourceExhaustedCause = 4 + // Workflow is busy + RESOURCE_EXHAUSTED_CAUSE_BUSY_WORKFLOW ResourceExhaustedCause = 5 + // Caller exceeds action per second limit. + RESOURCE_EXHAUSTED_CAUSE_APS_LIMIT ResourceExhaustedCause = 6 + // Persistence storage limit exceeded. + RESOURCE_EXHAUSTED_CAUSE_PERSISTENCE_STORAGE_LIMIT ResourceExhaustedCause = 7 + // Circuit breaker is open/half-open. + RESOURCE_EXHAUSTED_CAUSE_CIRCUIT_BREAKER_OPEN ResourceExhaustedCause = 8 +) + +// Enum value maps for ResourceExhaustedCause. +var ( + ResourceExhaustedCause_name = map[int32]string{ + 0: "RESOURCE_EXHAUSTED_CAUSE_UNSPECIFIED", + 1: "RESOURCE_EXHAUSTED_CAUSE_RPS_LIMIT", + 2: "RESOURCE_EXHAUSTED_CAUSE_CONCURRENT_LIMIT", + 3: "RESOURCE_EXHAUSTED_CAUSE_SYSTEM_OVERLOADED", + 4: "RESOURCE_EXHAUSTED_CAUSE_PERSISTENCE_LIMIT", + 5: "RESOURCE_EXHAUSTED_CAUSE_BUSY_WORKFLOW", + 6: "RESOURCE_EXHAUSTED_CAUSE_APS_LIMIT", + 7: "RESOURCE_EXHAUSTED_CAUSE_PERSISTENCE_STORAGE_LIMIT", + 8: "RESOURCE_EXHAUSTED_CAUSE_CIRCUIT_BREAKER_OPEN", + } + ResourceExhaustedCause_value = map[string]int32{ + "RESOURCE_EXHAUSTED_CAUSE_UNSPECIFIED": 0, + "RESOURCE_EXHAUSTED_CAUSE_RPS_LIMIT": 1, + "RESOURCE_EXHAUSTED_CAUSE_CONCURRENT_LIMIT": 2, + "RESOURCE_EXHAUSTED_CAUSE_SYSTEM_OVERLOADED": 3, + "RESOURCE_EXHAUSTED_CAUSE_PERSISTENCE_LIMIT": 4, + "RESOURCE_EXHAUSTED_CAUSE_BUSY_WORKFLOW": 5, + "RESOURCE_EXHAUSTED_CAUSE_APS_LIMIT": 6, + "RESOURCE_EXHAUSTED_CAUSE_PERSISTENCE_STORAGE_LIMIT": 7, + "RESOURCE_EXHAUSTED_CAUSE_CIRCUIT_BREAKER_OPEN": 8, + } +) + +func (x ResourceExhaustedCause) Enum() *ResourceExhaustedCause { + p := new(ResourceExhaustedCause) + *p = x + return p +} + +func (x ResourceExhaustedCause) String() string { + switch x { + case RESOURCE_EXHAUSTED_CAUSE_UNSPECIFIED: + return "Unspecified" + case RESOURCE_EXHAUSTED_CAUSE_RPS_LIMIT: + return "RpsLimit" + case RESOURCE_EXHAUSTED_CAUSE_CONCURRENT_LIMIT: + return "ConcurrentLimit" + case RESOURCE_EXHAUSTED_CAUSE_SYSTEM_OVERLOADED: + return "SystemOverloaded" + case RESOURCE_EXHAUSTED_CAUSE_PERSISTENCE_LIMIT: + return "PersistenceLimit" + case RESOURCE_EXHAUSTED_CAUSE_BUSY_WORKFLOW: + return "BusyWorkflow" + case RESOURCE_EXHAUSTED_CAUSE_APS_LIMIT: + return "ApsLimit" + case RESOURCE_EXHAUSTED_CAUSE_PERSISTENCE_STORAGE_LIMIT: + + // Deprecated: Use ResourceExhaustedCause.Descriptor instead. + return "PersistenceStorageLimit" + case RESOURCE_EXHAUSTED_CAUSE_CIRCUIT_BREAKER_OPEN: + return "CircuitBreakerOpen" + default: + return strconv.Itoa(int(x)) + } + +} + +func (ResourceExhaustedCause) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_failed_cause_proto_enumTypes[4].Descriptor() +} + +func (ResourceExhaustedCause) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_failed_cause_proto_enumTypes[4] +} + +func (x ResourceExhaustedCause) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +func (ResourceExhaustedCause) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_failed_cause_proto_rawDescGZIP(), []int{4} +} + +type ResourceExhaustedScope int32 + +const ( + RESOURCE_EXHAUSTED_SCOPE_UNSPECIFIED ResourceExhaustedScope = 0 + // Exhausted resource is a system-level resource. + RESOURCE_EXHAUSTED_SCOPE_NAMESPACE ResourceExhaustedScope = 1 + // Exhausted resource is a namespace-level resource. + RESOURCE_EXHAUSTED_SCOPE_SYSTEM ResourceExhaustedScope = 2 +) + +// Enum value maps for ResourceExhaustedScope. +var ( + ResourceExhaustedScope_name = map[int32]string{ + 0: "RESOURCE_EXHAUSTED_SCOPE_UNSPECIFIED", + 1: "RESOURCE_EXHAUSTED_SCOPE_NAMESPACE", + 2: "RESOURCE_EXHAUSTED_SCOPE_SYSTEM", + } + ResourceExhaustedScope_value = map[string]int32{ + "RESOURCE_EXHAUSTED_SCOPE_UNSPECIFIED": 0, + "RESOURCE_EXHAUSTED_SCOPE_NAMESPACE": 1, + "RESOURCE_EXHAUSTED_SCOPE_SYSTEM": 2, + } +) + +func (x ResourceExhaustedScope) Enum() *ResourceExhaustedScope { + p := new(ResourceExhaustedScope) + *p = x + return p +} + +func (x ResourceExhaustedScope) String() string { + switch x { + case RESOURCE_EXHAUSTED_SCOPE_UNSPECIFIED: + return "Unspecified" + case RESOURCE_EXHAUSTED_SCOPE_NAMESPACE: + return "Namespace" + case RESOURCE_EXHAUSTED_SCOPE_SYSTEM: + return "System" + default: + return strconv.Itoa(int(x)) + } + +} + +func (ResourceExhaustedScope) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_failed_cause_proto_enumTypes[5].Descriptor() +} + +func (ResourceExhaustedScope) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_failed_cause_proto_enumTypes[5] +} + +func (x ResourceExhaustedScope) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ResourceExhaustedScope.Descriptor instead. +func (ResourceExhaustedScope) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_failed_cause_proto_rawDescGZIP(), []int{5} +} + +var File_temporal_api_enums_v1_failed_cause_proto protoreflect.FileDescriptor + +var file_temporal_api_enums_v1_failed_cause_proto_rawDesc = []byte{ + 0x0a, 0x28, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x63, + 0x61, 0x75, 0x73, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, + 0x31, 0x2a, 0xee, 0x11, 0x0a, 0x17, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, + 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x61, 0x75, 0x73, 0x65, 0x12, 0x2a, 0x0a, + 0x26, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, + 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, + 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x30, 0x0a, 0x2c, 0x57, 0x4f, 0x52, + 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, + 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x55, 0x4e, 0x48, 0x41, 0x4e, 0x44, 0x4c, 0x45, + 0x44, 0x5f, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x10, 0x01, 0x12, 0x3f, 0x0a, 0x3b, 0x57, + 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, + 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x53, 0x43, + 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, + 0x41, 0x54, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, 0x02, 0x12, 0x45, 0x0a, 0x41, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, + 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x52, + 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x41, 0x43, + 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x41, 0x54, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, + 0x53, 0x10, 0x03, 0x12, 0x39, 0x0a, 0x35, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, + 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, + 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x54, 0x49, 0x4d, 0x45, + 0x52, 0x5f, 0x41, 0x54, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, 0x04, 0x12, 0x3a, + 0x0a, 0x36, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, + 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, + 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x52, 0x5f, 0x41, 0x54, + 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, 0x05, 0x12, 0x3b, 0x0a, 0x37, 0x57, 0x4f, + 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, + 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x52, 0x45, 0x43, + 0x4f, 0x52, 0x44, 0x5f, 0x4d, 0x41, 0x52, 0x4b, 0x45, 0x52, 0x5f, 0x41, 0x54, 0x54, 0x52, 0x49, + 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, 0x06, 0x12, 0x49, 0x0a, 0x45, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, + 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, + 0x54, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, + 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x54, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, + 0x10, 0x07, 0x12, 0x45, 0x0a, 0x41, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, + 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, + 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, + 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x54, 0x54, + 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, 0x08, 0x12, 0x47, 0x0a, 0x43, 0x57, 0x4f, 0x52, + 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, + 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x43, 0x41, 0x4e, 0x43, + 0x45, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, + 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x54, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, + 0x10, 0x09, 0x12, 0x58, 0x0a, 0x54, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, + 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, + 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x43, 0x41, 0x4e, + 0x43, 0x45, 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, + 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, + 0x41, 0x54, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, 0x0a, 0x12, 0x3d, 0x0a, 0x39, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, + 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x43, + 0x4f, 0x4e, 0x54, 0x49, 0x4e, 0x55, 0x45, 0x5f, 0x41, 0x53, 0x5f, 0x4e, 0x45, 0x57, 0x5f, 0x41, + 0x54, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, 0x0b, 0x12, 0x37, 0x0a, 0x33, 0x57, + 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, + 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, + 0x54, 0x49, 0x4d, 0x45, 0x52, 0x5f, 0x44, 0x55, 0x50, 0x4c, 0x49, 0x43, 0x41, 0x54, 0x45, 0x5f, + 0x49, 0x44, 0x10, 0x0c, 0x12, 0x36, 0x0a, 0x32, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, + 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, + 0x53, 0x45, 0x5f, 0x52, 0x45, 0x53, 0x45, 0x54, 0x5f, 0x53, 0x54, 0x49, 0x43, 0x4b, 0x59, 0x5f, + 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x51, 0x55, 0x45, 0x55, 0x45, 0x10, 0x0d, 0x12, 0x40, 0x0a, 0x3c, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, + 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x45, 0x52, 0x5f, 0x55, 0x4e, 0x48, 0x41, 0x4e, + 0x44, 0x4c, 0x45, 0x44, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x55, 0x52, 0x45, 0x10, 0x0e, 0x12, 0x47, + 0x0a, 0x43, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, + 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, + 0x5f, 0x53, 0x49, 0x47, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, + 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x54, 0x54, 0x52, 0x49, + 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, 0x0f, 0x12, 0x43, 0x0a, 0x3f, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, + 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, + 0x43, 0x48, 0x49, 0x4c, 0x44, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, + 0x41, 0x54, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, 0x10, 0x12, 0x32, 0x0a, 0x2e, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, + 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x46, 0x4f, 0x52, 0x43, 0x45, + 0x5f, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x5f, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x10, 0x11, + 0x12, 0x35, 0x0a, 0x31, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, + 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x46, + 0x41, 0x49, 0x4c, 0x4f, 0x56, 0x45, 0x52, 0x5f, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x5f, 0x43, 0x4f, + 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x10, 0x12, 0x12, 0x34, 0x0a, 0x30, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, + 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x53, 0x49, 0x47, 0x4e, 0x41, 0x4c, + 0x5f, 0x49, 0x4e, 0x50, 0x55, 0x54, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x10, 0x13, 0x12, 0x2d, 0x0a, + 0x29, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, + 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x52, 0x45, 0x53, 0x45, + 0x54, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x10, 0x14, 0x12, 0x29, 0x0a, 0x25, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, + 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x42, + 0x49, 0x4e, 0x41, 0x52, 0x59, 0x10, 0x15, 0x12, 0x3d, 0x0a, 0x39, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, + 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x5f, 0x41, + 0x43, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x44, 0x55, 0x50, 0x4c, 0x49, 0x43, 0x41, 0x54, + 0x45, 0x5f, 0x49, 0x44, 0x10, 0x16, 0x12, 0x34, 0x0a, 0x30, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, + 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, + 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x53, 0x45, 0x41, 0x52, 0x43, 0x48, 0x5f, + 0x41, 0x54, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, 0x17, 0x12, 0x36, 0x0a, 0x32, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, + 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x4e, 0x4f, 0x4e, 0x5f, 0x44, + 0x45, 0x54, 0x45, 0x52, 0x4d, 0x49, 0x4e, 0x49, 0x53, 0x54, 0x49, 0x43, 0x5f, 0x45, 0x52, 0x52, + 0x4f, 0x52, 0x10, 0x18, 0x12, 0x48, 0x0a, 0x44, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, + 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, + 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x59, 0x5f, 0x57, 0x4f, + 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x50, 0x52, 0x4f, 0x50, 0x45, 0x52, 0x54, 0x49, 0x45, + 0x53, 0x5f, 0x41, 0x54, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, 0x19, 0x12, 0x45, + 0x0a, 0x41, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, + 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x50, 0x45, 0x4e, + 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x48, 0x49, 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x53, 0x5f, 0x4c, 0x49, 0x4d, 0x49, 0x54, 0x5f, 0x45, 0x58, 0x43, 0x45, 0x45, + 0x44, 0x45, 0x44, 0x10, 0x1a, 0x12, 0x40, 0x0a, 0x3c, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, + 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, + 0x55, 0x53, 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x41, 0x43, 0x54, 0x49, + 0x56, 0x49, 0x54, 0x49, 0x45, 0x53, 0x5f, 0x4c, 0x49, 0x4d, 0x49, 0x54, 0x5f, 0x45, 0x58, 0x43, + 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x1b, 0x12, 0x3d, 0x0a, 0x39, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, + 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x53, 0x49, + 0x47, 0x4e, 0x41, 0x4c, 0x53, 0x5f, 0x4c, 0x49, 0x4d, 0x49, 0x54, 0x5f, 0x45, 0x58, 0x43, 0x45, + 0x45, 0x44, 0x45, 0x44, 0x10, 0x1c, 0x12, 0x44, 0x0a, 0x40, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, + 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, + 0x41, 0x55, 0x53, 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x52, 0x45, 0x51, + 0x55, 0x45, 0x53, 0x54, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x4c, 0x49, 0x4d, 0x49, + 0x54, 0x5f, 0x45, 0x58, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x1d, 0x12, 0x44, 0x0a, 0x40, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, + 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x55, + 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, + 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, + 0x10, 0x1e, 0x12, 0x2f, 0x0a, 0x2b, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, + 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, + 0x5f, 0x55, 0x4e, 0x48, 0x41, 0x4e, 0x44, 0x4c, 0x45, 0x44, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, + 0x45, 0x10, 0x1f, 0x12, 0x46, 0x0a, 0x42, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, + 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, + 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x5f, 0x4e, + 0x45, 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, + 0x54, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, 0x20, 0x12, 0x46, 0x0a, 0x42, 0x57, + 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, + 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, + 0x47, 0x5f, 0x4e, 0x45, 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, + 0x4e, 0x53, 0x5f, 0x4c, 0x49, 0x4d, 0x49, 0x54, 0x5f, 0x45, 0x58, 0x43, 0x45, 0x45, 0x44, 0x45, + 0x44, 0x10, 0x21, 0x12, 0x4c, 0x0a, 0x48, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, + 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, + 0x45, 0x5f, 0x42, 0x41, 0x44, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x43, 0x41, + 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x4e, 0x45, 0x58, 0x55, 0x53, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, + 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x54, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x53, 0x10, + 0x22, 0x12, 0x2f, 0x0a, 0x2b, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, + 0x53, 0x4b, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, + 0x46, 0x45, 0x41, 0x54, 0x55, 0x52, 0x45, 0x5f, 0x44, 0x49, 0x53, 0x41, 0x42, 0x4c, 0x45, 0x44, + 0x10, 0x23, 0x2a, 0xf3, 0x01, 0x0a, 0x26, 0x53, 0x74, 0x61, 0x72, 0x74, 0x43, 0x68, 0x69, 0x6c, + 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x61, 0x75, 0x73, 0x65, 0x12, 0x3b, 0x0a, + 0x37, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x43, 0x48, 0x49, 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, + 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, + 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x55, 0x4e, 0x53, + 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x47, 0x0a, 0x43, 0x53, 0x54, + 0x41, 0x52, 0x54, 0x5f, 0x43, 0x48, 0x49, 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, + 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, 0x41, 0x49, + 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, + 0x4f, 0x57, 0x5f, 0x41, 0x4c, 0x52, 0x45, 0x41, 0x44, 0x59, 0x5f, 0x45, 0x58, 0x49, 0x53, 0x54, + 0x53, 0x10, 0x01, 0x12, 0x43, 0x0a, 0x3f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x43, 0x48, 0x49, + 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, + 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, + 0x53, 0x45, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x5f, 0x4e, 0x4f, 0x54, + 0x5f, 0x46, 0x4f, 0x55, 0x4e, 0x44, 0x10, 0x02, 0x2a, 0x91, 0x02, 0x0a, 0x2a, 0x43, 0x61, 0x6e, + 0x63, 0x65, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, + 0x65, 0x64, 0x43, 0x61, 0x75, 0x73, 0x65, 0x12, 0x3f, 0x0a, 0x3b, 0x43, 0x41, 0x4e, 0x43, 0x45, + 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, 0x41, + 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, + 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x59, 0x0a, 0x55, 0x43, 0x41, 0x4e, 0x43, + 0x45, 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, + 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, + 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x45, 0x58, 0x54, 0x45, + 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, + 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x46, 0x4f, 0x55, 0x4e, + 0x44, 0x10, 0x01, 0x12, 0x47, 0x0a, 0x43, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x45, 0x58, + 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, + 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, + 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, + 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x46, 0x4f, 0x55, 0x4e, 0x44, 0x10, 0x02, 0x2a, 0xe2, 0x02, 0x0a, + 0x2a, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x61, 0x75, 0x73, 0x65, 0x12, 0x3f, 0x0a, 0x3b, 0x53, + 0x49, 0x47, 0x4e, 0x41, 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, + 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, + 0x4e, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x55, + 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x59, 0x0a, 0x55, + 0x53, 0x49, 0x47, 0x4e, 0x41, 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, + 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, + 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, + 0x46, 0x4f, 0x55, 0x4e, 0x44, 0x10, 0x01, 0x12, 0x47, 0x0a, 0x43, 0x53, 0x49, 0x47, 0x4e, 0x41, + 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, 0x41, + 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, + 0x50, 0x41, 0x43, 0x45, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x46, 0x4f, 0x55, 0x4e, 0x44, 0x10, 0x02, + 0x12, 0x4f, 0x0a, 0x4b, 0x53, 0x49, 0x47, 0x4e, 0x41, 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x52, + 0x4e, 0x41, 0x4c, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, + 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x43, 0x41, + 0x55, 0x53, 0x45, 0x5f, 0x53, 0x49, 0x47, 0x4e, 0x41, 0x4c, 0x5f, 0x43, 0x4f, 0x55, 0x4e, 0x54, + 0x5f, 0x4c, 0x49, 0x4d, 0x49, 0x54, 0x5f, 0x45, 0x58, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, + 0x03, 0x2a, 0xb8, 0x03, 0x0a, 0x16, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x78, + 0x68, 0x61, 0x75, 0x73, 0x74, 0x65, 0x64, 0x43, 0x61, 0x75, 0x73, 0x65, 0x12, 0x28, 0x0a, 0x24, + 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x58, 0x48, 0x41, 0x55, 0x53, 0x54, + 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, + 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x26, 0x0a, 0x22, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, + 0x43, 0x45, 0x5f, 0x45, 0x58, 0x48, 0x41, 0x55, 0x53, 0x54, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, + 0x53, 0x45, 0x5f, 0x52, 0x50, 0x53, 0x5f, 0x4c, 0x49, 0x4d, 0x49, 0x54, 0x10, 0x01, 0x12, 0x2d, + 0x0a, 0x29, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x58, 0x48, 0x41, 0x55, + 0x53, 0x54, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x43, 0x55, + 0x52, 0x52, 0x45, 0x4e, 0x54, 0x5f, 0x4c, 0x49, 0x4d, 0x49, 0x54, 0x10, 0x02, 0x12, 0x2e, 0x0a, + 0x2a, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x58, 0x48, 0x41, 0x55, 0x53, + 0x54, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x53, 0x59, 0x53, 0x54, 0x45, 0x4d, + 0x5f, 0x4f, 0x56, 0x45, 0x52, 0x4c, 0x4f, 0x41, 0x44, 0x45, 0x44, 0x10, 0x03, 0x12, 0x2e, 0x0a, + 0x2a, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x58, 0x48, 0x41, 0x55, 0x53, + 0x54, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, + 0x54, 0x45, 0x4e, 0x43, 0x45, 0x5f, 0x4c, 0x49, 0x4d, 0x49, 0x54, 0x10, 0x04, 0x12, 0x2a, 0x0a, + 0x26, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x58, 0x48, 0x41, 0x55, 0x53, + 0x54, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x42, 0x55, 0x53, 0x59, 0x5f, 0x57, + 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x10, 0x05, 0x12, 0x26, 0x0a, 0x22, 0x52, 0x45, 0x53, + 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x58, 0x48, 0x41, 0x55, 0x53, 0x54, 0x45, 0x44, 0x5f, + 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x41, 0x50, 0x53, 0x5f, 0x4c, 0x49, 0x4d, 0x49, 0x54, 0x10, + 0x06, 0x12, 0x36, 0x0a, 0x32, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x58, + 0x48, 0x41, 0x55, 0x53, 0x54, 0x45, 0x44, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x50, 0x45, + 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, 0x4e, 0x43, 0x45, 0x5f, 0x53, 0x54, 0x4f, 0x52, 0x41, 0x47, + 0x45, 0x5f, 0x4c, 0x49, 0x4d, 0x49, 0x54, 0x10, 0x07, 0x12, 0x31, 0x0a, 0x2d, 0x52, 0x45, 0x53, + 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x58, 0x48, 0x41, 0x55, 0x53, 0x54, 0x45, 0x44, 0x5f, + 0x43, 0x41, 0x55, 0x53, 0x45, 0x5f, 0x43, 0x49, 0x52, 0x43, 0x55, 0x49, 0x54, 0x5f, 0x42, 0x52, + 0x45, 0x41, 0x4b, 0x45, 0x52, 0x5f, 0x4f, 0x50, 0x45, 0x4e, 0x10, 0x08, 0x2a, 0x8f, 0x01, 0x0a, + 0x16, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x78, 0x68, 0x61, 0x75, 0x73, 0x74, + 0x65, 0x64, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x28, 0x0a, 0x24, 0x52, 0x45, 0x53, 0x4f, 0x55, + 0x52, 0x43, 0x45, 0x5f, 0x45, 0x58, 0x48, 0x41, 0x55, 0x53, 0x54, 0x45, 0x44, 0x5f, 0x53, 0x43, + 0x4f, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, + 0x00, 0x12, 0x26, 0x0a, 0x22, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x58, + 0x48, 0x41, 0x55, 0x53, 0x54, 0x45, 0x44, 0x5f, 0x53, 0x43, 0x4f, 0x50, 0x45, 0x5f, 0x4e, 0x41, + 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x10, 0x01, 0x12, 0x23, 0x0a, 0x1f, 0x52, 0x45, 0x53, + 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x58, 0x48, 0x41, 0x55, 0x53, 0x54, 0x45, 0x44, 0x5f, + 0x53, 0x43, 0x4f, 0x50, 0x45, 0x5f, 0x53, 0x59, 0x53, 0x54, 0x45, 0x4d, 0x10, 0x02, 0x42, 0x88, + 0x01, 0x0a, 0x18, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x42, 0x10, 0x46, 0x61, 0x69, + 0x6c, 0x65, 0x64, 0x43, 0x61, 0x75, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, + 0x21, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x3b, 0x65, 0x6e, 0x75, + 0x6d, 0x73, 0xaa, 0x02, 0x17, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, + 0x41, 0x70, 0x69, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1a, 0x54, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, + 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x33, +} + +var ( + file_temporal_api_enums_v1_failed_cause_proto_rawDescOnce sync.Once + file_temporal_api_enums_v1_failed_cause_proto_rawDescData = file_temporal_api_enums_v1_failed_cause_proto_rawDesc +) + +func file_temporal_api_enums_v1_failed_cause_proto_rawDescGZIP() []byte { + file_temporal_api_enums_v1_failed_cause_proto_rawDescOnce.Do(func() { + file_temporal_api_enums_v1_failed_cause_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_enums_v1_failed_cause_proto_rawDescData) + }) + return file_temporal_api_enums_v1_failed_cause_proto_rawDescData +} + +var file_temporal_api_enums_v1_failed_cause_proto_enumTypes = make([]protoimpl.EnumInfo, 6) +var file_temporal_api_enums_v1_failed_cause_proto_goTypes = []any{ + (WorkflowTaskFailedCause)(0), // 0: temporal.api.enums.v1.WorkflowTaskFailedCause + (StartChildWorkflowExecutionFailedCause)(0), // 1: temporal.api.enums.v1.StartChildWorkflowExecutionFailedCause + (CancelExternalWorkflowExecutionFailedCause)(0), // 2: temporal.api.enums.v1.CancelExternalWorkflowExecutionFailedCause + (SignalExternalWorkflowExecutionFailedCause)(0), // 3: temporal.api.enums.v1.SignalExternalWorkflowExecutionFailedCause + (ResourceExhaustedCause)(0), // 4: temporal.api.enums.v1.ResourceExhaustedCause + (ResourceExhaustedScope)(0), // 5: temporal.api.enums.v1.ResourceExhaustedScope +} +var file_temporal_api_enums_v1_failed_cause_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_enums_v1_failed_cause_proto_init() } +func file_temporal_api_enums_v1_failed_cause_proto_init() { + if File_temporal_api_enums_v1_failed_cause_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_enums_v1_failed_cause_proto_rawDesc, + NumEnums: 6, + NumMessages: 0, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_enums_v1_failed_cause_proto_goTypes, + DependencyIndexes: file_temporal_api_enums_v1_failed_cause_proto_depIdxs, + EnumInfos: file_temporal_api_enums_v1_failed_cause_proto_enumTypes, + }.Build() + File_temporal_api_enums_v1_failed_cause_proto = out.File + file_temporal_api_enums_v1_failed_cause_proto_rawDesc = nil + file_temporal_api_enums_v1_failed_cause_proto_goTypes = nil + file_temporal_api_enums_v1_failed_cause_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/enums/v1/namespace.go-helpers.pb.go b/vendor/go.temporal.io/api/enums/v1/namespace.go-helpers.pb.go new file mode 100644 index 00000000000..b03c4568399 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/namespace.go-helpers.pb.go @@ -0,0 +1,86 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package enums + +import ( + "fmt" +) + +var ( + NamespaceState_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Registered": 1, + "Deprecated": 2, + "Deleted": 3, + } +) + +// NamespaceStateFromString parses a NamespaceState value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to NamespaceState +func NamespaceStateFromString(s string) (NamespaceState, error) { + if v, ok := NamespaceState_value[s]; ok { + return NamespaceState(v), nil + } else if v, ok := NamespaceState_shorthandValue[s]; ok { + return NamespaceState(v), nil + } + return NamespaceState(0), fmt.Errorf("%s is not a valid NamespaceState", s) +} + +var ( + ArchivalState_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Disabled": 1, + "Enabled": 2, + } +) + +// ArchivalStateFromString parses a ArchivalState value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to ArchivalState +func ArchivalStateFromString(s string) (ArchivalState, error) { + if v, ok := ArchivalState_value[s]; ok { + return ArchivalState(v), nil + } else if v, ok := ArchivalState_shorthandValue[s]; ok { + return ArchivalState(v), nil + } + return ArchivalState(0), fmt.Errorf("%s is not a valid ArchivalState", s) +} + +var ( + ReplicationState_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Normal": 1, + "Handover": 2, + } +) + +// ReplicationStateFromString parses a ReplicationState value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to ReplicationState +func ReplicationStateFromString(s string) (ReplicationState, error) { + if v, ok := ReplicationState_value[s]; ok { + return ReplicationState(v), nil + } else if v, ok := ReplicationState_shorthandValue[s]; ok { + return ReplicationState(v), nil + } + return ReplicationState(0), fmt.Errorf("%s is not a valid ReplicationState", s) +} diff --git a/vendor/go.temporal.io/api/enums/v1/namespace.pb.go b/vendor/go.temporal.io/api/enums/v1/namespace.pb.go new file mode 100644 index 00000000000..db3d69116f9 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/namespace.pb.go @@ -0,0 +1,319 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/enums/v1/namespace.proto + +package enums + +import ( + reflect "reflect" + "strconv" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type NamespaceState int32 + +const ( + NAMESPACE_STATE_UNSPECIFIED NamespaceState = 0 + NAMESPACE_STATE_REGISTERED NamespaceState = 1 + NAMESPACE_STATE_DEPRECATED NamespaceState = 2 + NAMESPACE_STATE_DELETED NamespaceState = 3 +) + +// Enum value maps for NamespaceState. +var ( + NamespaceState_name = map[int32]string{ + 0: "NAMESPACE_STATE_UNSPECIFIED", + 1: "NAMESPACE_STATE_REGISTERED", + 2: "NAMESPACE_STATE_DEPRECATED", + 3: "NAMESPACE_STATE_DELETED", + } + NamespaceState_value = map[string]int32{ + "NAMESPACE_STATE_UNSPECIFIED": 0, + "NAMESPACE_STATE_REGISTERED": 1, + "NAMESPACE_STATE_DEPRECATED": 2, + "NAMESPACE_STATE_DELETED": 3, + } +) + +func (x NamespaceState) Enum() *NamespaceState { + p := new(NamespaceState) + *p = x + return p +} + +func (x NamespaceState) String() string { + switch x { + case NAMESPACE_STATE_UNSPECIFIED: + return "Unspecified" + case NAMESPACE_STATE_REGISTERED: + return "Registered" + case NAMESPACE_STATE_DEPRECATED: + return "Deprecated" + case NAMESPACE_STATE_DELETED: + return "Deleted" + default: + return strconv.Itoa(int(x)) + } + +} + +func (NamespaceState) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_namespace_proto_enumTypes[0].Descriptor() +} + +func (NamespaceState) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_namespace_proto_enumTypes[0] +} + +func (x NamespaceState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use NamespaceState.Descriptor instead. +func (NamespaceState) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_namespace_proto_rawDescGZIP(), []int{0} +} + +type ArchivalState int32 + +const ( + ARCHIVAL_STATE_UNSPECIFIED ArchivalState = 0 + ARCHIVAL_STATE_DISABLED ArchivalState = 1 + ARCHIVAL_STATE_ENABLED ArchivalState = 2 +) + +// Enum value maps for ArchivalState. +var ( + ArchivalState_name = map[int32]string{ + 0: "ARCHIVAL_STATE_UNSPECIFIED", + 1: "ARCHIVAL_STATE_DISABLED", + 2: "ARCHIVAL_STATE_ENABLED", + } + ArchivalState_value = map[string]int32{ + "ARCHIVAL_STATE_UNSPECIFIED": 0, + "ARCHIVAL_STATE_DISABLED": 1, + "ARCHIVAL_STATE_ENABLED": 2, + } +) + +func (x ArchivalState) Enum() *ArchivalState { + p := new(ArchivalState) + *p = x + return p +} + +func (x ArchivalState) String() string { + switch x { + case ARCHIVAL_STATE_UNSPECIFIED: + return "Unspecified" + case ARCHIVAL_STATE_DISABLED: + return "Disabled" + case ARCHIVAL_STATE_ENABLED: + return "Enabled" + default: + return strconv.Itoa(int(x)) + } + +} + +func (ArchivalState) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_namespace_proto_enumTypes[1].Descriptor() +} + +func (ArchivalState) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_namespace_proto_enumTypes[1] +} + +func (x ArchivalState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ArchivalState.Descriptor instead. +func (ArchivalState) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_namespace_proto_rawDescGZIP(), []int{1} +} + +type ReplicationState int32 + +const ( + REPLICATION_STATE_UNSPECIFIED ReplicationState = 0 + REPLICATION_STATE_NORMAL ReplicationState = 1 + REPLICATION_STATE_HANDOVER ReplicationState = 2 +) + +// Enum value maps for ReplicationState. +var ( + ReplicationState_name = map[int32]string{ + 0: "REPLICATION_STATE_UNSPECIFIED", + 1: "REPLICATION_STATE_NORMAL", + 2: "REPLICATION_STATE_HANDOVER", + } + ReplicationState_value = map[string]int32{ + "REPLICATION_STATE_UNSPECIFIED": 0, + "REPLICATION_STATE_NORMAL": 1, + "REPLICATION_STATE_HANDOVER": 2, + } +) + +func (x ReplicationState) Enum() *ReplicationState { + p := new(ReplicationState) + *p = x + return p +} + +func (x ReplicationState) String() string { + switch x { + case REPLICATION_STATE_UNSPECIFIED: + return "Unspecified" + case REPLICATION_STATE_NORMAL: + return "Normal" + case REPLICATION_STATE_HANDOVER: + return "Handover" + default: + return strconv.Itoa(int(x)) + } + +} + +func (ReplicationState) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_namespace_proto_enumTypes[2].Descriptor() +} + +func (ReplicationState) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_namespace_proto_enumTypes[2] +} + +func (x ReplicationState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ReplicationState.Descriptor instead. +func (ReplicationState) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_namespace_proto_rawDescGZIP(), []int{2} +} + +var File_temporal_api_enums_v1_namespace_proto protoreflect.FileDescriptor + +var file_temporal_api_enums_v1_namespace_proto_rawDesc = []byte{ + 0x0a, 0x25, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2a, 0x8e, + 0x01, 0x0a, 0x0e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x5f, 0x53, + 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, + 0x10, 0x00, 0x12, 0x1e, 0x0a, 0x1a, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x5f, + 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x52, 0x45, 0x47, 0x49, 0x53, 0x54, 0x45, 0x52, 0x45, 0x44, + 0x10, 0x01, 0x12, 0x1e, 0x0a, 0x1a, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x5f, + 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x44, 0x45, 0x50, 0x52, 0x45, 0x43, 0x41, 0x54, 0x45, 0x44, + 0x10, 0x02, 0x12, 0x1b, 0x0a, 0x17, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x5f, + 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x03, 0x2a, + 0x68, 0x0a, 0x0d, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x12, 0x1e, 0x0a, 0x1a, 0x41, 0x52, 0x43, 0x48, 0x49, 0x56, 0x41, 0x4c, 0x5f, 0x53, 0x54, 0x41, + 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, + 0x12, 0x1b, 0x0a, 0x17, 0x41, 0x52, 0x43, 0x48, 0x49, 0x56, 0x41, 0x4c, 0x5f, 0x53, 0x54, 0x41, + 0x54, 0x45, 0x5f, 0x44, 0x49, 0x53, 0x41, 0x42, 0x4c, 0x45, 0x44, 0x10, 0x01, 0x12, 0x1a, 0x0a, + 0x16, 0x41, 0x52, 0x43, 0x48, 0x49, 0x56, 0x41, 0x4c, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, + 0x45, 0x4e, 0x41, 0x42, 0x4c, 0x45, 0x44, 0x10, 0x02, 0x2a, 0x73, 0x0a, 0x10, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x21, 0x0a, + 0x1d, 0x52, 0x45, 0x50, 0x4c, 0x49, 0x43, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, + 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, + 0x12, 0x1c, 0x0a, 0x18, 0x52, 0x45, 0x50, 0x4c, 0x49, 0x43, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, + 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10, 0x01, 0x12, 0x1e, + 0x0a, 0x1a, 0x52, 0x45, 0x50, 0x4c, 0x49, 0x43, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, + 0x41, 0x54, 0x45, 0x5f, 0x48, 0x41, 0x4e, 0x44, 0x4f, 0x56, 0x45, 0x52, 0x10, 0x02, 0x42, 0x86, + 0x01, 0x0a, 0x18, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x42, 0x0e, 0x4e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x21, 0x67, + 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x3b, 0x65, 0x6e, 0x75, 0x6d, 0x73, + 0xaa, 0x02, 0x17, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, + 0x69, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1a, 0x54, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x45, 0x6e, + 0x75, 0x6d, 0x73, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_enums_v1_namespace_proto_rawDescOnce sync.Once + file_temporal_api_enums_v1_namespace_proto_rawDescData = file_temporal_api_enums_v1_namespace_proto_rawDesc +) + +func file_temporal_api_enums_v1_namespace_proto_rawDescGZIP() []byte { + file_temporal_api_enums_v1_namespace_proto_rawDescOnce.Do(func() { + file_temporal_api_enums_v1_namespace_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_enums_v1_namespace_proto_rawDescData) + }) + return file_temporal_api_enums_v1_namespace_proto_rawDescData +} + +var file_temporal_api_enums_v1_namespace_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_temporal_api_enums_v1_namespace_proto_goTypes = []any{ + (NamespaceState)(0), // 0: temporal.api.enums.v1.NamespaceState + (ArchivalState)(0), // 1: temporal.api.enums.v1.ArchivalState + (ReplicationState)(0), // 2: temporal.api.enums.v1.ReplicationState +} +var file_temporal_api_enums_v1_namespace_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_enums_v1_namespace_proto_init() } +func file_temporal_api_enums_v1_namespace_proto_init() { + if File_temporal_api_enums_v1_namespace_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_enums_v1_namespace_proto_rawDesc, + NumEnums: 3, + NumMessages: 0, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_enums_v1_namespace_proto_goTypes, + DependencyIndexes: file_temporal_api_enums_v1_namespace_proto_depIdxs, + EnumInfos: file_temporal_api_enums_v1_namespace_proto_enumTypes, + }.Build() + File_temporal_api_enums_v1_namespace_proto = out.File + file_temporal_api_enums_v1_namespace_proto_rawDesc = nil + file_temporal_api_enums_v1_namespace_proto_goTypes = nil + file_temporal_api_enums_v1_namespace_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/enums/v1/query.go-helpers.pb.go b/vendor/go.temporal.io/api/enums/v1/query.go-helpers.pb.go new file mode 100644 index 00000000000..d4e56fd7ee0 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/query.go-helpers.pb.go @@ -0,0 +1,67 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package enums + +import ( + "fmt" +) + +var ( + QueryResultType_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Answered": 1, + "Failed": 2, + } +) + +// QueryResultTypeFromString parses a QueryResultType value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to QueryResultType +func QueryResultTypeFromString(s string) (QueryResultType, error) { + if v, ok := QueryResultType_value[s]; ok { + return QueryResultType(v), nil + } else if v, ok := QueryResultType_shorthandValue[s]; ok { + return QueryResultType(v), nil + } + return QueryResultType(0), fmt.Errorf("%s is not a valid QueryResultType", s) +} + +var ( + QueryRejectCondition_shorthandValue = map[string]int32{ + "Unspecified": 0, + "None": 1, + "NotOpen": 2, + "NotCompletedCleanly": 3, + } +) + +// QueryRejectConditionFromString parses a QueryRejectCondition value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to QueryRejectCondition +func QueryRejectConditionFromString(s string) (QueryRejectCondition, error) { + if v, ok := QueryRejectCondition_value[s]; ok { + return QueryRejectCondition(v), nil + } else if v, ok := QueryRejectCondition_shorthandValue[s]; ok { + return QueryRejectCondition(v), nil + } + return QueryRejectCondition(0), fmt.Errorf("%s is not a valid QueryRejectCondition", s) +} diff --git a/vendor/go.temporal.io/api/enums/v1/query.pb.go b/vendor/go.temporal.io/api/enums/v1/query.pb.go new file mode 100644 index 00000000000..b07a8bbd74b --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/query.pb.go @@ -0,0 +1,258 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/enums/v1/query.proto + +package enums + +import ( + reflect "reflect" + "strconv" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type QueryResultType int32 + +const ( + QUERY_RESULT_TYPE_UNSPECIFIED QueryResultType = 0 + QUERY_RESULT_TYPE_ANSWERED QueryResultType = 1 + QUERY_RESULT_TYPE_FAILED QueryResultType = 2 +) + +// Enum value maps for QueryResultType. +var ( + QueryResultType_name = map[int32]string{ + 0: "QUERY_RESULT_TYPE_UNSPECIFIED", + 1: "QUERY_RESULT_TYPE_ANSWERED", + 2: "QUERY_RESULT_TYPE_FAILED", + } + QueryResultType_value = map[string]int32{ + "QUERY_RESULT_TYPE_UNSPECIFIED": 0, + "QUERY_RESULT_TYPE_ANSWERED": 1, + "QUERY_RESULT_TYPE_FAILED": 2, + } +) + +func (x QueryResultType) Enum() *QueryResultType { + p := new(QueryResultType) + *p = x + return p +} + +func (x QueryResultType) String() string { + switch x { + case QUERY_RESULT_TYPE_UNSPECIFIED: + return "Unspecified" + case QUERY_RESULT_TYPE_ANSWERED: + return "Answered" + case QUERY_RESULT_TYPE_FAILED: + return "Failed" + default: + return strconv.Itoa(int(x)) + } + +} + +func (QueryResultType) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_query_proto_enumTypes[0].Descriptor() +} + +func (QueryResultType) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_query_proto_enumTypes[0] +} + +func (x QueryResultType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use QueryResultType.Descriptor instead. +func (QueryResultType) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_query_proto_rawDescGZIP(), []int{0} +} + +type QueryRejectCondition int32 + +const ( + QUERY_REJECT_CONDITION_UNSPECIFIED QueryRejectCondition = 0 + // None indicates that query should not be rejected. + QUERY_REJECT_CONDITION_NONE QueryRejectCondition = 1 + // NotOpen indicates that query should be rejected if workflow is not open. + QUERY_REJECT_CONDITION_NOT_OPEN QueryRejectCondition = 2 + // NotCompletedCleanly indicates that query should be rejected if workflow did not complete cleanly. + QUERY_REJECT_CONDITION_NOT_COMPLETED_CLEANLY QueryRejectCondition = 3 +) + +// Enum value maps for QueryRejectCondition. +var ( + QueryRejectCondition_name = map[int32]string{ + 0: "QUERY_REJECT_CONDITION_UNSPECIFIED", + 1: "QUERY_REJECT_CONDITION_NONE", + 2: "QUERY_REJECT_CONDITION_NOT_OPEN", + 3: "QUERY_REJECT_CONDITION_NOT_COMPLETED_CLEANLY", + } + QueryRejectCondition_value = map[string]int32{ + "QUERY_REJECT_CONDITION_UNSPECIFIED": 0, + "QUERY_REJECT_CONDITION_NONE": 1, + "QUERY_REJECT_CONDITION_NOT_OPEN": 2, + "QUERY_REJECT_CONDITION_NOT_COMPLETED_CLEANLY": 3, + } +) + +func (x QueryRejectCondition) Enum() *QueryRejectCondition { + p := new(QueryRejectCondition) + *p = x + return p +} + +func (x QueryRejectCondition) String() string { + switch x { + case QUERY_REJECT_CONDITION_UNSPECIFIED: + return "Unspecified" + case QUERY_REJECT_CONDITION_NONE: + return "None" + case QUERY_REJECT_CONDITION_NOT_OPEN: + return "NotOpen" + case QUERY_REJECT_CONDITION_NOT_COMPLETED_CLEANLY: + return "NotCompletedCleanly" + default: + return strconv.Itoa(int(x)) + } + +} + +func (QueryRejectCondition) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_query_proto_enumTypes[1].Descriptor() +} + +func (QueryRejectCondition) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_query_proto_enumTypes[1] +} + +func (x QueryRejectCondition) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use QueryRejectCondition.Descriptor instead. +func (QueryRejectCondition) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_query_proto_rawDescGZIP(), []int{1} +} + +var File_temporal_api_enums_v1_query_proto protoreflect.FileDescriptor + +var file_temporal_api_enums_v1_query_proto_rawDesc = []byte{ + 0x0a, 0x21, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2a, 0x72, 0x0a, 0x0f, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x21, 0x0a, + 0x1d, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x54, 0x59, + 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, + 0x12, 0x1e, 0x0a, 0x1a, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x41, 0x4e, 0x53, 0x57, 0x45, 0x52, 0x45, 0x44, 0x10, 0x01, + 0x12, 0x1c, 0x0a, 0x18, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x02, 0x2a, 0xb6, + 0x01, 0x0a, 0x14, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x43, 0x6f, + 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x26, 0x0a, 0x22, 0x51, 0x55, 0x45, 0x52, 0x59, + 0x5f, 0x52, 0x45, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x43, 0x4f, 0x4e, 0x44, 0x49, 0x54, 0x49, 0x4f, + 0x4e, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, + 0x1f, 0x0a, 0x1b, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x52, 0x45, 0x4a, 0x45, 0x43, 0x54, 0x5f, + 0x43, 0x4f, 0x4e, 0x44, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x01, + 0x12, 0x23, 0x0a, 0x1f, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x52, 0x45, 0x4a, 0x45, 0x43, 0x54, + 0x5f, 0x43, 0x4f, 0x4e, 0x44, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x4f, + 0x50, 0x45, 0x4e, 0x10, 0x02, 0x12, 0x30, 0x0a, 0x2c, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x52, + 0x45, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x43, 0x4f, 0x4e, 0x44, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, + 0x4e, 0x4f, 0x54, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x5f, 0x43, 0x4c, + 0x45, 0x41, 0x4e, 0x4c, 0x59, 0x10, 0x03, 0x42, 0x82, 0x01, 0x0a, 0x18, 0x69, 0x6f, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, + 0x73, 0x2e, 0x76, 0x31, 0x42, 0x0a, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x6f, 0x74, 0x6f, + 0x50, 0x01, 0x5a, 0x21, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x3b, + 0x65, 0x6e, 0x75, 0x6d, 0x73, 0xaa, 0x02, 0x17, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x56, 0x31, 0xea, + 0x02, 0x1a, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, + 0x69, 0x3a, 0x3a, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_enums_v1_query_proto_rawDescOnce sync.Once + file_temporal_api_enums_v1_query_proto_rawDescData = file_temporal_api_enums_v1_query_proto_rawDesc +) + +func file_temporal_api_enums_v1_query_proto_rawDescGZIP() []byte { + file_temporal_api_enums_v1_query_proto_rawDescOnce.Do(func() { + file_temporal_api_enums_v1_query_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_enums_v1_query_proto_rawDescData) + }) + return file_temporal_api_enums_v1_query_proto_rawDescData +} + +var file_temporal_api_enums_v1_query_proto_enumTypes = make([]protoimpl.EnumInfo, 2) +var file_temporal_api_enums_v1_query_proto_goTypes = []any{ + (QueryResultType)(0), // 0: temporal.api.enums.v1.QueryResultType + (QueryRejectCondition)(0), // 1: temporal.api.enums.v1.QueryRejectCondition +} +var file_temporal_api_enums_v1_query_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_enums_v1_query_proto_init() } +func file_temporal_api_enums_v1_query_proto_init() { + if File_temporal_api_enums_v1_query_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_enums_v1_query_proto_rawDesc, + NumEnums: 2, + NumMessages: 0, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_enums_v1_query_proto_goTypes, + DependencyIndexes: file_temporal_api_enums_v1_query_proto_depIdxs, + EnumInfos: file_temporal_api_enums_v1_query_proto_enumTypes, + }.Build() + File_temporal_api_enums_v1_query_proto = out.File + file_temporal_api_enums_v1_query_proto_rawDesc = nil + file_temporal_api_enums_v1_query_proto_goTypes = nil + file_temporal_api_enums_v1_query_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/enums/v1/reset.go-helpers.pb.go b/vendor/go.temporal.io/api/enums/v1/reset.go-helpers.pb.go new file mode 100644 index 00000000000..f7629a91f3d --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/reset.go-helpers.pb.go @@ -0,0 +1,86 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package enums + +import ( + "fmt" +) + +var ( + ResetReapplyExcludeType_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Signal": 1, + "Update": 2, + } +) + +// ResetReapplyExcludeTypeFromString parses a ResetReapplyExcludeType value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to ResetReapplyExcludeType +func ResetReapplyExcludeTypeFromString(s string) (ResetReapplyExcludeType, error) { + if v, ok := ResetReapplyExcludeType_value[s]; ok { + return ResetReapplyExcludeType(v), nil + } else if v, ok := ResetReapplyExcludeType_shorthandValue[s]; ok { + return ResetReapplyExcludeType(v), nil + } + return ResetReapplyExcludeType(0), fmt.Errorf("%s is not a valid ResetReapplyExcludeType", s) +} + +var ( + ResetReapplyType_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Signal": 1, + "None": 2, + "AllEligible": 3, + } +) + +// ResetReapplyTypeFromString parses a ResetReapplyType value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to ResetReapplyType +func ResetReapplyTypeFromString(s string) (ResetReapplyType, error) { + if v, ok := ResetReapplyType_value[s]; ok { + return ResetReapplyType(v), nil + } else if v, ok := ResetReapplyType_shorthandValue[s]; ok { + return ResetReapplyType(v), nil + } + return ResetReapplyType(0), fmt.Errorf("%s is not a valid ResetReapplyType", s) +} + +var ( + ResetType_shorthandValue = map[string]int32{ + "Unspecified": 0, + "FirstWorkflowTask": 1, + "LastWorkflowTask": 2, + } +) + +// ResetTypeFromString parses a ResetType value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to ResetType +func ResetTypeFromString(s string) (ResetType, error) { + if v, ok := ResetType_value[s]; ok { + return ResetType(v), nil + } else if v, ok := ResetType_shorthandValue[s]; ok { + return ResetType(v), nil + } + return ResetType(0), fmt.Errorf("%s is not a valid ResetType", s) +} diff --git a/vendor/go.temporal.io/api/enums/v1/reset.pb.go b/vendor/go.temporal.io/api/enums/v1/reset.pb.go new file mode 100644 index 00000000000..5b22dac9246 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/reset.pb.go @@ -0,0 +1,334 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/enums/v1/reset.proto + +package enums + +import ( + reflect "reflect" + "strconv" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// Event types to exclude when reapplying events. +type ResetReapplyExcludeType int32 + +const ( + RESET_REAPPLY_EXCLUDE_TYPE_UNSPECIFIED ResetReapplyExcludeType = 0 + // Exclude signals when reapplying events. + RESET_REAPPLY_EXCLUDE_TYPE_SIGNAL ResetReapplyExcludeType = 1 + // Exclude updates when reapplying events. + RESET_REAPPLY_EXCLUDE_TYPE_UPDATE ResetReapplyExcludeType = 2 +) + +// Enum value maps for ResetReapplyExcludeType. +var ( + ResetReapplyExcludeType_name = map[int32]string{ + 0: "RESET_REAPPLY_EXCLUDE_TYPE_UNSPECIFIED", + 1: "RESET_REAPPLY_EXCLUDE_TYPE_SIGNAL", + 2: "RESET_REAPPLY_EXCLUDE_TYPE_UPDATE", + } + ResetReapplyExcludeType_value = map[string]int32{ + "RESET_REAPPLY_EXCLUDE_TYPE_UNSPECIFIED": 0, + "RESET_REAPPLY_EXCLUDE_TYPE_SIGNAL": 1, + "RESET_REAPPLY_EXCLUDE_TYPE_UPDATE": 2, + } +) + +func (x ResetReapplyExcludeType) Enum() *ResetReapplyExcludeType { + p := new(ResetReapplyExcludeType) + *p = x + return p +} + +func (x ResetReapplyExcludeType) String() string { + switch x { + case RESET_REAPPLY_EXCLUDE_TYPE_UNSPECIFIED: + return "Unspecified" + case RESET_REAPPLY_EXCLUDE_TYPE_SIGNAL: + return "Signal" + case RESET_REAPPLY_EXCLUDE_TYPE_UPDATE: + return "Update" + default: + return strconv.Itoa(int(x)) + } + +} + +func (ResetReapplyExcludeType) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_reset_proto_enumTypes[0].Descriptor() +} + +func (ResetReapplyExcludeType) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_reset_proto_enumTypes[0] +} + +func (x ResetReapplyExcludeType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ResetReapplyExcludeType.Descriptor instead. +func (ResetReapplyExcludeType) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_reset_proto_rawDescGZIP(), []int{0} +} + +// Event types to include when reapplying events. Deprecated: applications +// should use ResetReapplyExcludeType to specify exclusions from this set, and +// new event types should be added to ResetReapplyExcludeType instead of here. +type ResetReapplyType int32 + +const ( + RESET_REAPPLY_TYPE_UNSPECIFIED ResetReapplyType = 0 + // Signals are reapplied when workflow is reset. + RESET_REAPPLY_TYPE_SIGNAL ResetReapplyType = 1 + // No events are reapplied when workflow is reset. + RESET_REAPPLY_TYPE_NONE ResetReapplyType = 2 + // All eligible events are reapplied when workflow is reset. + RESET_REAPPLY_TYPE_ALL_ELIGIBLE ResetReapplyType = 3 +) + +// Enum value maps for ResetReapplyType. +var ( + ResetReapplyType_name = map[int32]string{ + 0: "RESET_REAPPLY_TYPE_UNSPECIFIED", + 1: "RESET_REAPPLY_TYPE_SIGNAL", + 2: "RESET_REAPPLY_TYPE_NONE", + 3: "RESET_REAPPLY_TYPE_ALL_ELIGIBLE", + } + ResetReapplyType_value = map[string]int32{ + "RESET_REAPPLY_TYPE_UNSPECIFIED": 0, + "RESET_REAPPLY_TYPE_SIGNAL": 1, + "RESET_REAPPLY_TYPE_NONE": 2, + "RESET_REAPPLY_TYPE_ALL_ELIGIBLE": 3, + } +) + +func (x ResetReapplyType) Enum() *ResetReapplyType { + p := new(ResetReapplyType) + *p = x + return p +} + +func (x ResetReapplyType) String() string { + switch x { + case RESET_REAPPLY_TYPE_UNSPECIFIED: + return "Unspecified" + case RESET_REAPPLY_TYPE_SIGNAL: + return "Signal" + case RESET_REAPPLY_TYPE_NONE: + return "None" + case RESET_REAPPLY_TYPE_ALL_ELIGIBLE: + return "AllEligible" + default: + return strconv.Itoa(int(x)) + } + +} + +func (ResetReapplyType) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_reset_proto_enumTypes[1].Descriptor() +} + +func (ResetReapplyType) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_reset_proto_enumTypes[1] +} + +func (x ResetReapplyType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ResetReapplyType.Descriptor instead. +func (ResetReapplyType) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_reset_proto_rawDescGZIP(), []int{1} +} + +// Reset type options. Deprecated, see temporal.api.common.v1.ResetOptions. +type ResetType int32 + +const ( + RESET_TYPE_UNSPECIFIED ResetType = 0 + // Resets to event of the first workflow task completed, or if it does not exist, the event after task scheduled. + RESET_TYPE_FIRST_WORKFLOW_TASK ResetType = 1 + // Resets to event of the last workflow task completed, or if it does not exist, the event after task scheduled. + RESET_TYPE_LAST_WORKFLOW_TASK ResetType = 2 +) + +// Enum value maps for ResetType. +var ( + ResetType_name = map[int32]string{ + 0: "RESET_TYPE_UNSPECIFIED", + 1: "RESET_TYPE_FIRST_WORKFLOW_TASK", + 2: "RESET_TYPE_LAST_WORKFLOW_TASK", + } + ResetType_value = map[string]int32{ + "RESET_TYPE_UNSPECIFIED": 0, + "RESET_TYPE_FIRST_WORKFLOW_TASK": 1, + "RESET_TYPE_LAST_WORKFLOW_TASK": 2, + } +) + +func (x ResetType) Enum() *ResetType { + p := new(ResetType) + *p = x + return p +} + +func (x ResetType) String() string { + switch x { + case RESET_TYPE_UNSPECIFIED: + return "Unspecified" + case RESET_TYPE_FIRST_WORKFLOW_TASK: + return "FirstWorkflowTask" + case RESET_TYPE_LAST_WORKFLOW_TASK: + return "LastWorkflowTask" + default: + return strconv.Itoa(int(x)) + } + +} + +func (ResetType) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_reset_proto_enumTypes[2].Descriptor() +} + +func (ResetType) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_reset_proto_enumTypes[2] +} + +func (x ResetType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ResetType.Descriptor instead. +func (ResetType) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_reset_proto_rawDescGZIP(), []int{2} +} + +var File_temporal_api_enums_v1_reset_proto protoreflect.FileDescriptor + +var file_temporal_api_enums_v1_reset_proto_rawDesc = []byte{ + 0x0a, 0x21, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x72, 0x65, 0x73, 0x65, 0x74, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2a, 0x93, 0x01, 0x0a, 0x17, 0x52, + 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x45, 0x78, 0x63, 0x6c, 0x75, + 0x64, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x2a, 0x0a, 0x26, 0x52, 0x45, 0x53, 0x45, 0x54, 0x5f, + 0x52, 0x45, 0x41, 0x50, 0x50, 0x4c, 0x59, 0x5f, 0x45, 0x58, 0x43, 0x4c, 0x55, 0x44, 0x45, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, + 0x10, 0x00, 0x12, 0x25, 0x0a, 0x21, 0x52, 0x45, 0x53, 0x45, 0x54, 0x5f, 0x52, 0x45, 0x41, 0x50, + 0x50, 0x4c, 0x59, 0x5f, 0x45, 0x58, 0x43, 0x4c, 0x55, 0x44, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, + 0x5f, 0x53, 0x49, 0x47, 0x4e, 0x41, 0x4c, 0x10, 0x01, 0x12, 0x25, 0x0a, 0x21, 0x52, 0x45, 0x53, + 0x45, 0x54, 0x5f, 0x52, 0x45, 0x41, 0x50, 0x50, 0x4c, 0x59, 0x5f, 0x45, 0x58, 0x43, 0x4c, 0x55, + 0x44, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x02, + 0x2a, 0x97, 0x01, 0x0a, 0x10, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x61, 0x70, 0x70, 0x6c, + 0x79, 0x54, 0x79, 0x70, 0x65, 0x12, 0x22, 0x0a, 0x1e, 0x52, 0x45, 0x53, 0x45, 0x54, 0x5f, 0x52, + 0x45, 0x41, 0x50, 0x50, 0x4c, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, + 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52, 0x45, 0x53, + 0x45, 0x54, 0x5f, 0x52, 0x45, 0x41, 0x50, 0x50, 0x4c, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, + 0x53, 0x49, 0x47, 0x4e, 0x41, 0x4c, 0x10, 0x01, 0x12, 0x1b, 0x0a, 0x17, 0x52, 0x45, 0x53, 0x45, + 0x54, 0x5f, 0x52, 0x45, 0x41, 0x50, 0x50, 0x4c, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4e, + 0x4f, 0x4e, 0x45, 0x10, 0x02, 0x12, 0x23, 0x0a, 0x1f, 0x52, 0x45, 0x53, 0x45, 0x54, 0x5f, 0x52, + 0x45, 0x41, 0x50, 0x50, 0x4c, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x41, 0x4c, 0x4c, 0x5f, + 0x45, 0x4c, 0x49, 0x47, 0x49, 0x42, 0x4c, 0x45, 0x10, 0x03, 0x2a, 0x6e, 0x0a, 0x09, 0x52, 0x65, + 0x73, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x52, 0x45, 0x53, 0x45, 0x54, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, + 0x44, 0x10, 0x00, 0x12, 0x22, 0x0a, 0x1e, 0x52, 0x45, 0x53, 0x45, 0x54, 0x5f, 0x54, 0x59, 0x50, + 0x45, 0x5f, 0x46, 0x49, 0x52, 0x53, 0x54, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, + 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53, 0x45, 0x54, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4c, 0x41, 0x53, 0x54, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x10, 0x02, 0x42, 0x82, 0x01, 0x0a, 0x18, 0x69, + 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x42, 0x0a, 0x52, 0x65, 0x73, 0x65, 0x74, 0x50, 0x72, + 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x21, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, + 0x76, 0x31, 0x3b, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0xaa, 0x02, 0x17, 0x54, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x56, 0x31, 0xea, 0x02, 0x1a, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, + 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x3a, 0x3a, 0x56, 0x31, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_enums_v1_reset_proto_rawDescOnce sync.Once + file_temporal_api_enums_v1_reset_proto_rawDescData = file_temporal_api_enums_v1_reset_proto_rawDesc +) + +func file_temporal_api_enums_v1_reset_proto_rawDescGZIP() []byte { + file_temporal_api_enums_v1_reset_proto_rawDescOnce.Do(func() { + file_temporal_api_enums_v1_reset_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_enums_v1_reset_proto_rawDescData) + }) + return file_temporal_api_enums_v1_reset_proto_rawDescData +} + +var file_temporal_api_enums_v1_reset_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_temporal_api_enums_v1_reset_proto_goTypes = []any{ + (ResetReapplyExcludeType)(0), // 0: temporal.api.enums.v1.ResetReapplyExcludeType + (ResetReapplyType)(0), // 1: temporal.api.enums.v1.ResetReapplyType + (ResetType)(0), // 2: temporal.api.enums.v1.ResetType +} +var file_temporal_api_enums_v1_reset_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_enums_v1_reset_proto_init() } +func file_temporal_api_enums_v1_reset_proto_init() { + if File_temporal_api_enums_v1_reset_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_enums_v1_reset_proto_rawDesc, + NumEnums: 3, + NumMessages: 0, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_enums_v1_reset_proto_goTypes, + DependencyIndexes: file_temporal_api_enums_v1_reset_proto_depIdxs, + EnumInfos: file_temporal_api_enums_v1_reset_proto_enumTypes, + }.Build() + File_temporal_api_enums_v1_reset_proto = out.File + file_temporal_api_enums_v1_reset_proto_rawDesc = nil + file_temporal_api_enums_v1_reset_proto_goTypes = nil + file_temporal_api_enums_v1_reset_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/enums/v1/schedule.go-helpers.pb.go b/vendor/go.temporal.io/api/enums/v1/schedule.go-helpers.pb.go new file mode 100644 index 00000000000..01fdfa498b0 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/schedule.go-helpers.pb.go @@ -0,0 +1,51 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package enums + +import ( + "fmt" +) + +var ( + ScheduleOverlapPolicy_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Skip": 1, + "BufferOne": 2, + "BufferAll": 3, + "CancelOther": 4, + "TerminateOther": 5, + "AllowAll": 6, + } +) + +// ScheduleOverlapPolicyFromString parses a ScheduleOverlapPolicy value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to ScheduleOverlapPolicy +func ScheduleOverlapPolicyFromString(s string) (ScheduleOverlapPolicy, error) { + if v, ok := ScheduleOverlapPolicy_value[s]; ok { + return ScheduleOverlapPolicy(v), nil + } else if v, ok := ScheduleOverlapPolicy_shorthandValue[s]; ok { + return ScheduleOverlapPolicy(v), nil + } + return ScheduleOverlapPolicy(0), fmt.Errorf("%s is not a valid ScheduleOverlapPolicy", s) +} diff --git a/vendor/go.temporal.io/api/enums/v1/schedule.pb.go b/vendor/go.temporal.io/api/enums/v1/schedule.pb.go new file mode 100644 index 00000000000..754251a76f5 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/schedule.pb.go @@ -0,0 +1,227 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/enums/v1/schedule.proto + +package enums + +import ( + reflect "reflect" + "strconv" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// ScheduleOverlapPolicy controls what happens when a workflow would be started +// by a schedule, and is already running. +type ScheduleOverlapPolicy int32 + +const ( + SCHEDULE_OVERLAP_POLICY_UNSPECIFIED ScheduleOverlapPolicy = 0 + // SCHEDULE_OVERLAP_POLICY_SKIP (default) means don't start anything. When the + // workflow completes, the next scheduled event after that time will be considered. + SCHEDULE_OVERLAP_POLICY_SKIP ScheduleOverlapPolicy = 1 + // SCHEDULE_OVERLAP_POLICY_BUFFER_ONE means start the workflow again soon as the + // current one completes, but only buffer one start in this way. If another start is + // supposed to happen when the workflow is running, and one is already buffered, then + // only the first one will be started after the running workflow finishes. + SCHEDULE_OVERLAP_POLICY_BUFFER_ONE ScheduleOverlapPolicy = 2 + // SCHEDULE_OVERLAP_POLICY_BUFFER_ALL means buffer up any number of starts to all + // happen sequentially, immediately after the running workflow completes. + SCHEDULE_OVERLAP_POLICY_BUFFER_ALL ScheduleOverlapPolicy = 3 + // SCHEDULE_OVERLAP_POLICY_CANCEL_OTHER means that if there is another workflow + // running, cancel it, and start the new one after the old one completes cancellation. + SCHEDULE_OVERLAP_POLICY_CANCEL_OTHER ScheduleOverlapPolicy = 4 + // SCHEDULE_OVERLAP_POLICY_TERMINATE_OTHER means that if there is another workflow + // running, terminate it and start the new one immediately. + SCHEDULE_OVERLAP_POLICY_TERMINATE_OTHER ScheduleOverlapPolicy = 5 + // SCHEDULE_OVERLAP_POLICY_ALLOW_ALL means start any number of concurrent workflows. + // Note that with this policy, last completion result and last failure will not be + // available since workflows are not sequential. + SCHEDULE_OVERLAP_POLICY_ALLOW_ALL ScheduleOverlapPolicy = 6 +) + +// Enum value maps for ScheduleOverlapPolicy. +var ( + ScheduleOverlapPolicy_name = map[int32]string{ + 0: "SCHEDULE_OVERLAP_POLICY_UNSPECIFIED", + 1: "SCHEDULE_OVERLAP_POLICY_SKIP", + 2: "SCHEDULE_OVERLAP_POLICY_BUFFER_ONE", + 3: "SCHEDULE_OVERLAP_POLICY_BUFFER_ALL", + 4: "SCHEDULE_OVERLAP_POLICY_CANCEL_OTHER", + 5: "SCHEDULE_OVERLAP_POLICY_TERMINATE_OTHER", + 6: "SCHEDULE_OVERLAP_POLICY_ALLOW_ALL", + } + ScheduleOverlapPolicy_value = map[string]int32{ + "SCHEDULE_OVERLAP_POLICY_UNSPECIFIED": 0, + "SCHEDULE_OVERLAP_POLICY_SKIP": 1, + "SCHEDULE_OVERLAP_POLICY_BUFFER_ONE": 2, + "SCHEDULE_OVERLAP_POLICY_BUFFER_ALL": 3, + "SCHEDULE_OVERLAP_POLICY_CANCEL_OTHER": 4, + "SCHEDULE_OVERLAP_POLICY_TERMINATE_OTHER": 5, + "SCHEDULE_OVERLAP_POLICY_ALLOW_ALL": 6, + } +) + +func (x ScheduleOverlapPolicy) Enum() *ScheduleOverlapPolicy { + p := new(ScheduleOverlapPolicy) + *p = x + return p +} + +func (x ScheduleOverlapPolicy) String() string { + switch x { + case SCHEDULE_OVERLAP_POLICY_UNSPECIFIED: + return "Unspecified" + case SCHEDULE_OVERLAP_POLICY_SKIP: + return "Skip" + case SCHEDULE_OVERLAP_POLICY_BUFFER_ONE: + return "BufferOne" + case SCHEDULE_OVERLAP_POLICY_BUFFER_ALL: + return "BufferAll" + case SCHEDULE_OVERLAP_POLICY_CANCEL_OTHER: + return "CancelOther" + case SCHEDULE_OVERLAP_POLICY_TERMINATE_OTHER: + return "TerminateOther" + case SCHEDULE_OVERLAP_POLICY_ALLOW_ALL: + return "AllowAll" + default: + return strconv.Itoa(int(x)) + } + +} + +func (ScheduleOverlapPolicy) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_schedule_proto_enumTypes[0].Descriptor() +} + +func (ScheduleOverlapPolicy) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_schedule_proto_enumTypes[0] +} + +func (x ScheduleOverlapPolicy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ScheduleOverlapPolicy.Descriptor instead. +func (ScheduleOverlapPolicy) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_schedule_proto_rawDescGZIP(), []int{0} +} + +var File_temporal_api_enums_v1_schedule_proto protoreflect.FileDescriptor + +var file_temporal_api_enums_v1_schedule_proto_rawDesc = []byte{ + 0x0a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2a, 0xb0, 0x02, + 0x0a, 0x15, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4f, 0x76, 0x65, 0x72, 0x6c, 0x61, + 0x70, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x27, 0x0a, 0x23, 0x53, 0x43, 0x48, 0x45, 0x44, + 0x55, 0x4c, 0x45, 0x5f, 0x4f, 0x56, 0x45, 0x52, 0x4c, 0x41, 0x50, 0x5f, 0x50, 0x4f, 0x4c, 0x49, + 0x43, 0x59, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, + 0x12, 0x20, 0x0a, 0x1c, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x5f, 0x4f, 0x56, 0x45, + 0x52, 0x4c, 0x41, 0x50, 0x5f, 0x50, 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x53, 0x4b, 0x49, 0x50, + 0x10, 0x01, 0x12, 0x26, 0x0a, 0x22, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x5f, 0x4f, + 0x56, 0x45, 0x52, 0x4c, 0x41, 0x50, 0x5f, 0x50, 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x42, 0x55, + 0x46, 0x46, 0x45, 0x52, 0x5f, 0x4f, 0x4e, 0x45, 0x10, 0x02, 0x12, 0x26, 0x0a, 0x22, 0x53, 0x43, + 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x5f, 0x4f, 0x56, 0x45, 0x52, 0x4c, 0x41, 0x50, 0x5f, 0x50, + 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x42, 0x55, 0x46, 0x46, 0x45, 0x52, 0x5f, 0x41, 0x4c, 0x4c, + 0x10, 0x03, 0x12, 0x28, 0x0a, 0x24, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x5f, 0x4f, + 0x56, 0x45, 0x52, 0x4c, 0x41, 0x50, 0x5f, 0x50, 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x43, 0x41, + 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x4f, 0x54, 0x48, 0x45, 0x52, 0x10, 0x04, 0x12, 0x2b, 0x0a, 0x27, + 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x5f, 0x4f, 0x56, 0x45, 0x52, 0x4c, 0x41, 0x50, + 0x5f, 0x50, 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x49, 0x4e, 0x41, 0x54, + 0x45, 0x5f, 0x4f, 0x54, 0x48, 0x45, 0x52, 0x10, 0x05, 0x12, 0x25, 0x0a, 0x21, 0x53, 0x43, 0x48, + 0x45, 0x44, 0x55, 0x4c, 0x45, 0x5f, 0x4f, 0x56, 0x45, 0x52, 0x4c, 0x41, 0x50, 0x5f, 0x50, 0x4f, + 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x41, 0x4c, 0x4c, 0x4f, 0x57, 0x5f, 0x41, 0x4c, 0x4c, 0x10, 0x06, + 0x42, 0x85, 0x01, 0x0a, 0x18, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x42, 0x0d, 0x53, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x21, + 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, + 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x3b, 0x65, 0x6e, 0x75, 0x6d, + 0x73, 0xaa, 0x02, 0x17, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, + 0x70, 0x69, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1a, 0x54, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x45, + 0x6e, 0x75, 0x6d, 0x73, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_enums_v1_schedule_proto_rawDescOnce sync.Once + file_temporal_api_enums_v1_schedule_proto_rawDescData = file_temporal_api_enums_v1_schedule_proto_rawDesc +) + +func file_temporal_api_enums_v1_schedule_proto_rawDescGZIP() []byte { + file_temporal_api_enums_v1_schedule_proto_rawDescOnce.Do(func() { + file_temporal_api_enums_v1_schedule_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_enums_v1_schedule_proto_rawDescData) + }) + return file_temporal_api_enums_v1_schedule_proto_rawDescData +} + +var file_temporal_api_enums_v1_schedule_proto_enumTypes = make([]protoimpl.EnumInfo, 1) +var file_temporal_api_enums_v1_schedule_proto_goTypes = []any{ + (ScheduleOverlapPolicy)(0), // 0: temporal.api.enums.v1.ScheduleOverlapPolicy +} +var file_temporal_api_enums_v1_schedule_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_enums_v1_schedule_proto_init() } +func file_temporal_api_enums_v1_schedule_proto_init() { + if File_temporal_api_enums_v1_schedule_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_enums_v1_schedule_proto_rawDesc, + NumEnums: 1, + NumMessages: 0, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_enums_v1_schedule_proto_goTypes, + DependencyIndexes: file_temporal_api_enums_v1_schedule_proto_depIdxs, + EnumInfos: file_temporal_api_enums_v1_schedule_proto_enumTypes, + }.Build() + File_temporal_api_enums_v1_schedule_proto = out.File + file_temporal_api_enums_v1_schedule_proto_rawDesc = nil + file_temporal_api_enums_v1_schedule_proto_goTypes = nil + file_temporal_api_enums_v1_schedule_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/enums/v1/task_queue.go-helpers.pb.go b/vendor/go.temporal.io/api/enums/v1/task_queue.go-helpers.pb.go new file mode 100644 index 00000000000..e161f3cf5a9 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/task_queue.go-helpers.pb.go @@ -0,0 +1,126 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package enums + +import ( + "fmt" +) + +var ( + TaskQueueKind_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Normal": 1, + "Sticky": 2, + } +) + +// TaskQueueKindFromString parses a TaskQueueKind value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to TaskQueueKind +func TaskQueueKindFromString(s string) (TaskQueueKind, error) { + if v, ok := TaskQueueKind_value[s]; ok { + return TaskQueueKind(v), nil + } else if v, ok := TaskQueueKind_shorthandValue[s]; ok { + return TaskQueueKind(v), nil + } + return TaskQueueKind(0), fmt.Errorf("%s is not a valid TaskQueueKind", s) +} + +var ( + TaskQueueType_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Workflow": 1, + "Activity": 2, + "Nexus": 3, + } +) + +// TaskQueueTypeFromString parses a TaskQueueType value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to TaskQueueType +func TaskQueueTypeFromString(s string) (TaskQueueType, error) { + if v, ok := TaskQueueType_value[s]; ok { + return TaskQueueType(v), nil + } else if v, ok := TaskQueueType_shorthandValue[s]; ok { + return TaskQueueType(v), nil + } + return TaskQueueType(0), fmt.Errorf("%s is not a valid TaskQueueType", s) +} + +var ( + TaskReachability_shorthandValue = map[string]int32{ + "Unspecified": 0, + "NewWorkflows": 1, + "ExistingWorkflows": 2, + "OpenWorkflows": 3, + "ClosedWorkflows": 4, + } +) + +// TaskReachabilityFromString parses a TaskReachability value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to TaskReachability +func TaskReachabilityFromString(s string) (TaskReachability, error) { + if v, ok := TaskReachability_value[s]; ok { + return TaskReachability(v), nil + } else if v, ok := TaskReachability_shorthandValue[s]; ok { + return TaskReachability(v), nil + } + return TaskReachability(0), fmt.Errorf("%s is not a valid TaskReachability", s) +} + +var ( + BuildIdTaskReachability_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Reachable": 1, + "ClosedWorkflowsOnly": 2, + "Unreachable": 3, + } +) + +// BuildIdTaskReachabilityFromString parses a BuildIdTaskReachability value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to BuildIdTaskReachability +func BuildIdTaskReachabilityFromString(s string) (BuildIdTaskReachability, error) { + if v, ok := BuildIdTaskReachability_value[s]; ok { + return BuildIdTaskReachability(v), nil + } else if v, ok := BuildIdTaskReachability_shorthandValue[s]; ok { + return BuildIdTaskReachability(v), nil + } + return BuildIdTaskReachability(0), fmt.Errorf("%s is not a valid BuildIdTaskReachability", s) +} + +var ( + DescribeTaskQueueMode_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Enhanced": 1, + } +) + +// DescribeTaskQueueModeFromString parses a DescribeTaskQueueMode value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to DescribeTaskQueueMode +func DescribeTaskQueueModeFromString(s string) (DescribeTaskQueueMode, error) { + if v, ok := DescribeTaskQueueMode_value[s]; ok { + return DescribeTaskQueueMode(v), nil + } else if v, ok := DescribeTaskQueueMode_shorthandValue[s]; ok { + return DescribeTaskQueueMode(v), nil + } + return DescribeTaskQueueMode(0), fmt.Errorf("%s is not a valid DescribeTaskQueueMode", s) +} diff --git a/vendor/go.temporal.io/api/enums/v1/task_queue.pb.go b/vendor/go.temporal.io/api/enums/v1/task_queue.pb.go new file mode 100644 index 00000000000..4844f1a8aa6 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/task_queue.pb.go @@ -0,0 +1,523 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/enums/v1/task_queue.proto + +package enums + +import ( + reflect "reflect" + "strconv" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type TaskQueueKind int32 + +const ( + TASK_QUEUE_KIND_UNSPECIFIED TaskQueueKind = 0 + // Tasks from a normal workflow task queue always include complete workflow history + // + // The task queue specified by the user is always a normal task queue. There can be as many + // workers as desired for a single normal task queue. All those workers may pick up tasks from + // that queue. + TASK_QUEUE_KIND_NORMAL TaskQueueKind = 1 + // A sticky queue only includes new history since the last workflow task, and they are + // per-worker. + // + // Sticky queues are created dynamically by each worker during their start up. They only exist + // for the lifetime of the worker process. Tasks in a sticky task queue are only available to + // the worker that created the sticky queue. + // + // Sticky queues are only for workflow tasks. There are no sticky task queues for activities. + TASK_QUEUE_KIND_STICKY TaskQueueKind = 2 +) + +// Enum value maps for TaskQueueKind. +var ( + TaskQueueKind_name = map[int32]string{ + 0: "TASK_QUEUE_KIND_UNSPECIFIED", + 1: "TASK_QUEUE_KIND_NORMAL", + 2: "TASK_QUEUE_KIND_STICKY", + } + TaskQueueKind_value = map[string]int32{ + "TASK_QUEUE_KIND_UNSPECIFIED": 0, + "TASK_QUEUE_KIND_NORMAL": 1, + "TASK_QUEUE_KIND_STICKY": 2, + } +) + +func (x TaskQueueKind) Enum() *TaskQueueKind { + p := new(TaskQueueKind) + *p = x + return p +} + +func (x TaskQueueKind) String() string { + switch x { + case TASK_QUEUE_KIND_UNSPECIFIED: + return "Unspecified" + case TASK_QUEUE_KIND_NORMAL: + return "Normal" + case TASK_QUEUE_KIND_STICKY: + return "Sticky" + default: + return strconv.Itoa(int(x)) + } + +} + +func (TaskQueueKind) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_task_queue_proto_enumTypes[0].Descriptor() +} + +func (TaskQueueKind) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_task_queue_proto_enumTypes[0] +} + +func (x TaskQueueKind) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use TaskQueueKind.Descriptor instead. +func (TaskQueueKind) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_task_queue_proto_rawDescGZIP(), []int{0} +} + +type TaskQueueType int32 + +const ( + TASK_QUEUE_TYPE_UNSPECIFIED TaskQueueType = 0 + // Workflow type of task queue. + TASK_QUEUE_TYPE_WORKFLOW TaskQueueType = 1 + // Activity type of task queue. + TASK_QUEUE_TYPE_ACTIVITY TaskQueueType = 2 + // Task queue type for dispatching Nexus requests. + TASK_QUEUE_TYPE_NEXUS TaskQueueType = 3 +) + +// Enum value maps for TaskQueueType. +var ( + TaskQueueType_name = map[int32]string{ + 0: "TASK_QUEUE_TYPE_UNSPECIFIED", + 1: "TASK_QUEUE_TYPE_WORKFLOW", + 2: "TASK_QUEUE_TYPE_ACTIVITY", + 3: "TASK_QUEUE_TYPE_NEXUS", + } + TaskQueueType_value = map[string]int32{ + "TASK_QUEUE_TYPE_UNSPECIFIED": 0, + "TASK_QUEUE_TYPE_WORKFLOW": 1, + "TASK_QUEUE_TYPE_ACTIVITY": 2, + "TASK_QUEUE_TYPE_NEXUS": 3, + } +) + +func (x TaskQueueType) Enum() *TaskQueueType { + p := new(TaskQueueType) + *p = x + return p +} + +func (x TaskQueueType) String() string { + switch x { + case TASK_QUEUE_TYPE_UNSPECIFIED: + return "Unspecified" + case TASK_QUEUE_TYPE_WORKFLOW: + return "Workflow" + case TASK_QUEUE_TYPE_ACTIVITY: + return "Activity" + case TASK_QUEUE_TYPE_NEXUS: + return "Nexus" + default: + return strconv.Itoa(int(x)) + } + +} + +func (TaskQueueType) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_task_queue_proto_enumTypes[1].Descriptor() +} + +func (TaskQueueType) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_task_queue_proto_enumTypes[1] +} + +func (x TaskQueueType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use TaskQueueType.Descriptor instead. +func (TaskQueueType) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_task_queue_proto_rawDescGZIP(), []int{1} +} + +// Specifies which category of tasks may reach a worker on a versioned task queue. +// Used both in a reachability query and its response. +// Deprecated. +type TaskReachability int32 + +const ( + TASK_REACHABILITY_UNSPECIFIED TaskReachability = 0 + // There's a possiblity for a worker to receive new workflow tasks. Workers should *not* be retired. + TASK_REACHABILITY_NEW_WORKFLOWS TaskReachability = 1 + // There's a possiblity for a worker to receive existing workflow and activity tasks from existing workflows. Workers + // should *not* be retired. + // This enum value does not distinguish between open and closed workflows. + TASK_REACHABILITY_EXISTING_WORKFLOWS TaskReachability = 2 + // There's a possiblity for a worker to receive existing workflow and activity tasks from open workflows. Workers + // should *not* be retired. + TASK_REACHABILITY_OPEN_WORKFLOWS TaskReachability = 3 + // There's a possiblity for a worker to receive existing workflow tasks from closed workflows. Workers may be + // retired dependending on application requirements. For example, if there's no need to query closed workflows. + TASK_REACHABILITY_CLOSED_WORKFLOWS TaskReachability = 4 +) + +// Enum value maps for TaskReachability. +var ( + TaskReachability_name = map[int32]string{ + 0: "TASK_REACHABILITY_UNSPECIFIED", + 1: "TASK_REACHABILITY_NEW_WORKFLOWS", + 2: "TASK_REACHABILITY_EXISTING_WORKFLOWS", + 3: "TASK_REACHABILITY_OPEN_WORKFLOWS", + 4: "TASK_REACHABILITY_CLOSED_WORKFLOWS", + } + TaskReachability_value = map[string]int32{ + "TASK_REACHABILITY_UNSPECIFIED": 0, + "TASK_REACHABILITY_NEW_WORKFLOWS": 1, + "TASK_REACHABILITY_EXISTING_WORKFLOWS": 2, + "TASK_REACHABILITY_OPEN_WORKFLOWS": 3, + "TASK_REACHABILITY_CLOSED_WORKFLOWS": 4, + } +) + +func (x TaskReachability) Enum() *TaskReachability { + p := new(TaskReachability) + *p = x + return p +} + +func (x TaskReachability) String() string { + switch x { + case TASK_REACHABILITY_UNSPECIFIED: + return "Unspecified" + case TASK_REACHABILITY_NEW_WORKFLOWS: + return "NewWorkflows" + case TASK_REACHABILITY_EXISTING_WORKFLOWS: + return "ExistingWorkflows" + case TASK_REACHABILITY_OPEN_WORKFLOWS: + return "OpenWorkflows" + case TASK_REACHABILITY_CLOSED_WORKFLOWS: + return "ClosedWorkflows" + default: + return strconv.Itoa(int(x)) + } + +} + +func (TaskReachability) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_task_queue_proto_enumTypes[2].Descriptor() +} + +func (TaskReachability) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_task_queue_proto_enumTypes[2] +} + +func (x TaskReachability) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use TaskReachability.Descriptor instead. +func (TaskReachability) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_task_queue_proto_rawDescGZIP(), []int{2} +} + +// Specifies which category of tasks may reach a versioned worker of a certain Build ID. +// +// Task Reachability is eventually consistent; there may be a delay (up to few minutes) until it +// converges to the most accurate value but it is designed in a way to take the more conservative +// side until it converges. For example REACHABLE is more conservative than CLOSED_WORKFLOWS_ONLY. +// +// Note: future activities who inherit their workflow's Build ID but not its Task Queue will not be +// accounted for reachability as server cannot know if they'll happen as they do not use +// assignment rules of their Task Queue. Same goes for Child Workflows or Continue-As-New Workflows +// who inherit the parent/previous workflow's Build ID but not its Task Queue. In those cases, make +// sure to query reachability for the parent/previous workflow's Task Queue as well. +type BuildIdTaskReachability int32 + +const ( + // Task reachability is not reported + BUILD_ID_TASK_REACHABILITY_UNSPECIFIED BuildIdTaskReachability = 0 + // Build ID may be used by new workflows or activities (base on versioning rules), or there MAY + // be open workflows or backlogged activities assigned to it. + BUILD_ID_TASK_REACHABILITY_REACHABLE BuildIdTaskReachability = 1 + // Build ID does not have open workflows and is not reachable by new workflows, + // but MAY have closed workflows within the namespace retention period. + // Not applicable to activity-only task queues. + BUILD_ID_TASK_REACHABILITY_CLOSED_WORKFLOWS_ONLY BuildIdTaskReachability = 2 + // Build ID is not used for new executions, nor it has been used by any existing execution + // within the retention period. + BUILD_ID_TASK_REACHABILITY_UNREACHABLE BuildIdTaskReachability = 3 +) + +// Enum value maps for BuildIdTaskReachability. +var ( + BuildIdTaskReachability_name = map[int32]string{ + 0: "BUILD_ID_TASK_REACHABILITY_UNSPECIFIED", + 1: "BUILD_ID_TASK_REACHABILITY_REACHABLE", + 2: "BUILD_ID_TASK_REACHABILITY_CLOSED_WORKFLOWS_ONLY", + 3: "BUILD_ID_TASK_REACHABILITY_UNREACHABLE", + } + BuildIdTaskReachability_value = map[string]int32{ + "BUILD_ID_TASK_REACHABILITY_UNSPECIFIED": 0, + "BUILD_ID_TASK_REACHABILITY_REACHABLE": 1, + "BUILD_ID_TASK_REACHABILITY_CLOSED_WORKFLOWS_ONLY": 2, + "BUILD_ID_TASK_REACHABILITY_UNREACHABLE": 3, + } +) + +func (x BuildIdTaskReachability) Enum() *BuildIdTaskReachability { + p := new(BuildIdTaskReachability) + *p = x + return p +} + +func (x BuildIdTaskReachability) String() string { + switch x { + case BUILD_ID_TASK_REACHABILITY_UNSPECIFIED: + return "Unspecified" + case BUILD_ID_TASK_REACHABILITY_REACHABLE: + return "Reachable" + case BUILD_ID_TASK_REACHABILITY_CLOSED_WORKFLOWS_ONLY: + return "ClosedWorkflowsOnly" + case BUILD_ID_TASK_REACHABILITY_UNREACHABLE: + return "Unreachable" + default: + return strconv.Itoa(int(x)) + } + +} + +func (BuildIdTaskReachability) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_task_queue_proto_enumTypes[3].Descriptor() +} + +func (BuildIdTaskReachability) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_task_queue_proto_enumTypes[3] +} + +func (x BuildIdTaskReachability) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use BuildIdTaskReachability.Descriptor instead. +func (BuildIdTaskReachability) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_task_queue_proto_rawDescGZIP(), []int{3} +} + +type DescribeTaskQueueMode int32 + +const ( + // Unspecified means legacy behavior. + DESCRIBE_TASK_QUEUE_MODE_UNSPECIFIED DescribeTaskQueueMode = 0 + // Enhanced mode reports aggregated results for all partitions, supports Build IDs, and reports richer info. + DESCRIBE_TASK_QUEUE_MODE_ENHANCED DescribeTaskQueueMode = 1 +) + +// Enum value maps for DescribeTaskQueueMode. +var ( + DescribeTaskQueueMode_name = map[int32]string{ + 0: "DESCRIBE_TASK_QUEUE_MODE_UNSPECIFIED", + 1: "DESCRIBE_TASK_QUEUE_MODE_ENHANCED", + } + DescribeTaskQueueMode_value = map[string]int32{ + "DESCRIBE_TASK_QUEUE_MODE_UNSPECIFIED": 0, + "DESCRIBE_TASK_QUEUE_MODE_ENHANCED": 1, + } +) + +func (x DescribeTaskQueueMode) Enum() *DescribeTaskQueueMode { + p := new(DescribeTaskQueueMode) + *p = x + return p +} + +func (x DescribeTaskQueueMode) String() string { + switch x { + case DESCRIBE_TASK_QUEUE_MODE_UNSPECIFIED: + return "Unspecified" + case DESCRIBE_TASK_QUEUE_MODE_ENHANCED: + return "Enhanced" + default: + return strconv.Itoa(int(x)) + } + +} + +func (DescribeTaskQueueMode) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_task_queue_proto_enumTypes[4].Descriptor() +} + +func (DescribeTaskQueueMode) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_task_queue_proto_enumTypes[4] +} + +func (x DescribeTaskQueueMode) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use DescribeTaskQueueMode.Descriptor instead. +func (DescribeTaskQueueMode) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_task_queue_proto_rawDescGZIP(), []int{4} +} + +var File_temporal_api_enums_v1_task_queue_proto protoreflect.FileDescriptor + +var file_temporal_api_enums_v1_task_queue_proto_rawDesc = []byte{ + 0x0a, 0x26, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, + 0x75, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2a, + 0x68, 0x0a, 0x0d, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x4b, 0x69, 0x6e, 0x64, + 0x12, 0x1f, 0x0a, 0x1b, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x51, 0x55, 0x45, 0x55, 0x45, 0x5f, 0x4b, + 0x49, 0x4e, 0x44, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, + 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x51, 0x55, 0x45, 0x55, 0x45, 0x5f, + 0x4b, 0x49, 0x4e, 0x44, 0x5f, 0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10, 0x01, 0x12, 0x1a, 0x0a, + 0x16, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x51, 0x55, 0x45, 0x55, 0x45, 0x5f, 0x4b, 0x49, 0x4e, 0x44, + 0x5f, 0x53, 0x54, 0x49, 0x43, 0x4b, 0x59, 0x10, 0x02, 0x2a, 0x87, 0x01, 0x0a, 0x0d, 0x54, 0x61, + 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x54, + 0x41, 0x53, 0x4b, 0x5f, 0x51, 0x55, 0x45, 0x55, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, + 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x1c, 0x0a, 0x18, + 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x51, 0x55, 0x45, 0x55, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x10, 0x01, 0x12, 0x1c, 0x0a, 0x18, 0x54, 0x41, + 0x53, 0x4b, 0x5f, 0x51, 0x55, 0x45, 0x55, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x41, 0x43, + 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x10, 0x02, 0x12, 0x19, 0x0a, 0x15, 0x54, 0x41, 0x53, 0x4b, + 0x5f, 0x51, 0x55, 0x45, 0x55, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4e, 0x45, 0x58, 0x55, + 0x53, 0x10, 0x03, 0x2a, 0xd2, 0x01, 0x0a, 0x10, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x61, 0x63, + 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x12, 0x21, 0x0a, 0x1d, 0x54, 0x41, 0x53, 0x4b, + 0x5f, 0x52, 0x45, 0x41, 0x43, 0x48, 0x41, 0x42, 0x49, 0x4c, 0x49, 0x54, 0x59, 0x5f, 0x55, 0x4e, + 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x23, 0x0a, 0x1f, 0x54, + 0x41, 0x53, 0x4b, 0x5f, 0x52, 0x45, 0x41, 0x43, 0x48, 0x41, 0x42, 0x49, 0x4c, 0x49, 0x54, 0x59, + 0x5f, 0x4e, 0x45, 0x57, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x53, 0x10, 0x01, + 0x12, 0x28, 0x0a, 0x24, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x52, 0x45, 0x41, 0x43, 0x48, 0x41, 0x42, + 0x49, 0x4c, 0x49, 0x54, 0x59, 0x5f, 0x45, 0x58, 0x49, 0x53, 0x54, 0x49, 0x4e, 0x47, 0x5f, 0x57, + 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x53, 0x10, 0x02, 0x12, 0x24, 0x0a, 0x20, 0x54, 0x41, + 0x53, 0x4b, 0x5f, 0x52, 0x45, 0x41, 0x43, 0x48, 0x41, 0x42, 0x49, 0x4c, 0x49, 0x54, 0x59, 0x5f, + 0x4f, 0x50, 0x45, 0x4e, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x53, 0x10, 0x03, + 0x12, 0x26, 0x0a, 0x22, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x52, 0x45, 0x41, 0x43, 0x48, 0x41, 0x42, + 0x49, 0x4c, 0x49, 0x54, 0x59, 0x5f, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x44, 0x5f, 0x57, 0x4f, 0x52, + 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x53, 0x10, 0x04, 0x2a, 0xd1, 0x01, 0x0a, 0x17, 0x42, 0x75, 0x69, + 0x6c, 0x64, 0x49, 0x64, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x12, 0x2a, 0x0a, 0x26, 0x42, 0x55, 0x49, 0x4c, 0x44, 0x5f, 0x49, 0x44, + 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x52, 0x45, 0x41, 0x43, 0x48, 0x41, 0x42, 0x49, 0x4c, 0x49, + 0x54, 0x59, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, + 0x12, 0x28, 0x0a, 0x24, 0x42, 0x55, 0x49, 0x4c, 0x44, 0x5f, 0x49, 0x44, 0x5f, 0x54, 0x41, 0x53, + 0x4b, 0x5f, 0x52, 0x45, 0x41, 0x43, 0x48, 0x41, 0x42, 0x49, 0x4c, 0x49, 0x54, 0x59, 0x5f, 0x52, + 0x45, 0x41, 0x43, 0x48, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x01, 0x12, 0x34, 0x0a, 0x30, 0x42, 0x55, + 0x49, 0x4c, 0x44, 0x5f, 0x49, 0x44, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x52, 0x45, 0x41, 0x43, + 0x48, 0x41, 0x42, 0x49, 0x4c, 0x49, 0x54, 0x59, 0x5f, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x44, 0x5f, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x53, 0x5f, 0x4f, 0x4e, 0x4c, 0x59, 0x10, 0x02, + 0x12, 0x2a, 0x0a, 0x26, 0x42, 0x55, 0x49, 0x4c, 0x44, 0x5f, 0x49, 0x44, 0x5f, 0x54, 0x41, 0x53, + 0x4b, 0x5f, 0x52, 0x45, 0x41, 0x43, 0x48, 0x41, 0x42, 0x49, 0x4c, 0x49, 0x54, 0x59, 0x5f, 0x55, + 0x4e, 0x52, 0x45, 0x41, 0x43, 0x48, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x03, 0x2a, 0x68, 0x0a, 0x15, + 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, + 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x28, 0x0a, 0x24, 0x44, 0x45, 0x53, 0x43, 0x52, 0x49, 0x42, + 0x45, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x51, 0x55, 0x45, 0x55, 0x45, 0x5f, 0x4d, 0x4f, 0x44, + 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, + 0x25, 0x0a, 0x21, 0x44, 0x45, 0x53, 0x43, 0x52, 0x49, 0x42, 0x45, 0x5f, 0x54, 0x41, 0x53, 0x4b, + 0x5f, 0x51, 0x55, 0x45, 0x55, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x45, 0x4e, 0x48, 0x41, + 0x4e, 0x43, 0x45, 0x44, 0x10, 0x01, 0x42, 0x86, 0x01, 0x0a, 0x18, 0x69, 0x6f, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, + 0x2e, 0x76, 0x31, 0x42, 0x0e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x50, 0x72, + 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x21, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, + 0x76, 0x31, 0x3b, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0xaa, 0x02, 0x17, 0x54, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x56, 0x31, 0xea, 0x02, 0x1a, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, + 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x3a, 0x3a, 0x56, 0x31, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_enums_v1_task_queue_proto_rawDescOnce sync.Once + file_temporal_api_enums_v1_task_queue_proto_rawDescData = file_temporal_api_enums_v1_task_queue_proto_rawDesc +) + +func file_temporal_api_enums_v1_task_queue_proto_rawDescGZIP() []byte { + file_temporal_api_enums_v1_task_queue_proto_rawDescOnce.Do(func() { + file_temporal_api_enums_v1_task_queue_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_enums_v1_task_queue_proto_rawDescData) + }) + return file_temporal_api_enums_v1_task_queue_proto_rawDescData +} + +var file_temporal_api_enums_v1_task_queue_proto_enumTypes = make([]protoimpl.EnumInfo, 5) +var file_temporal_api_enums_v1_task_queue_proto_goTypes = []any{ + (TaskQueueKind)(0), // 0: temporal.api.enums.v1.TaskQueueKind + (TaskQueueType)(0), // 1: temporal.api.enums.v1.TaskQueueType + (TaskReachability)(0), // 2: temporal.api.enums.v1.TaskReachability + (BuildIdTaskReachability)(0), // 3: temporal.api.enums.v1.BuildIdTaskReachability + (DescribeTaskQueueMode)(0), // 4: temporal.api.enums.v1.DescribeTaskQueueMode +} +var file_temporal_api_enums_v1_task_queue_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_enums_v1_task_queue_proto_init() } +func file_temporal_api_enums_v1_task_queue_proto_init() { + if File_temporal_api_enums_v1_task_queue_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_enums_v1_task_queue_proto_rawDesc, + NumEnums: 5, + NumMessages: 0, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_enums_v1_task_queue_proto_goTypes, + DependencyIndexes: file_temporal_api_enums_v1_task_queue_proto_depIdxs, + EnumInfos: file_temporal_api_enums_v1_task_queue_proto_enumTypes, + }.Build() + File_temporal_api_enums_v1_task_queue_proto = out.File + file_temporal_api_enums_v1_task_queue_proto_rawDesc = nil + file_temporal_api_enums_v1_task_queue_proto_goTypes = nil + file_temporal_api_enums_v1_task_queue_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/enums/v1/update.go-helpers.pb.go b/vendor/go.temporal.io/api/enums/v1/update.go-helpers.pb.go new file mode 100644 index 00000000000..a2c8ff08b7e --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/update.go-helpers.pb.go @@ -0,0 +1,66 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package enums + +import ( + "fmt" +) + +var ( + UpdateWorkflowExecutionLifecycleStage_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Admitted": 1, + "Accepted": 2, + "Completed": 3, + } +) + +// UpdateWorkflowExecutionLifecycleStageFromString parses a UpdateWorkflowExecutionLifecycleStage value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to UpdateWorkflowExecutionLifecycleStage +func UpdateWorkflowExecutionLifecycleStageFromString(s string) (UpdateWorkflowExecutionLifecycleStage, error) { + if v, ok := UpdateWorkflowExecutionLifecycleStage_value[s]; ok { + return UpdateWorkflowExecutionLifecycleStage(v), nil + } else if v, ok := UpdateWorkflowExecutionLifecycleStage_shorthandValue[s]; ok { + return UpdateWorkflowExecutionLifecycleStage(v), nil + } + return UpdateWorkflowExecutionLifecycleStage(0), fmt.Errorf("%s is not a valid UpdateWorkflowExecutionLifecycleStage", s) +} + +var ( + UpdateAdmittedEventOrigin_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Reapply": 1, + } +) + +// UpdateAdmittedEventOriginFromString parses a UpdateAdmittedEventOrigin value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to UpdateAdmittedEventOrigin +func UpdateAdmittedEventOriginFromString(s string) (UpdateAdmittedEventOrigin, error) { + if v, ok := UpdateAdmittedEventOrigin_value[s]; ok { + return UpdateAdmittedEventOrigin(v), nil + } else if v, ok := UpdateAdmittedEventOrigin_shorthandValue[s]; ok { + return UpdateAdmittedEventOrigin(v), nil + } + return UpdateAdmittedEventOrigin(0), fmt.Errorf("%s is not a valid UpdateAdmittedEventOrigin", s) +} diff --git a/vendor/go.temporal.io/api/enums/v1/update.pb.go b/vendor/go.temporal.io/api/enums/v1/update.pb.go new file mode 100644 index 00000000000..8d82bb587c8 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/update.pb.go @@ -0,0 +1,277 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/enums/v1/update.proto + +package enums + +import ( + reflect "reflect" + "strconv" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// UpdateWorkflowExecutionLifecycleStage is specified by clients invoking +// workflow execution updates and used to indicate to the server how long the +// client wishes to wait for a return value from the RPC. If any value other +// than UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED is sent by the +// client then the RPC will complete before the update is finished and will +// return a handle to the running update so that it can later be polled for +// completion. +type UpdateWorkflowExecutionLifecycleStage int32 + +const ( + // An unspecified vale for this enum. + UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED UpdateWorkflowExecutionLifecycleStage = 0 + // The gRPC call will not return until the update request has been admitted + // by the server - it may be the case that due to a considerations like load + // or resource limits that an update is made to wait before the server will + // indicate that it has been received and will be processed. This value + // does not wait for any sort of acknowledgement from a worker. + UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ADMITTED UpdateWorkflowExecutionLifecycleStage = 1 + // The gRPC call will not return until the update has passed validation on + // a worker. + UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ACCEPTED UpdateWorkflowExecutionLifecycleStage = 2 + // The gRPC call will not return until the update has executed to completion + // on a worker and has either been rejected or returned a value or an error. + UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED UpdateWorkflowExecutionLifecycleStage = 3 +) + +// Enum value maps for UpdateWorkflowExecutionLifecycleStage. +var ( + UpdateWorkflowExecutionLifecycleStage_name = map[int32]string{ + 0: "UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED", + 1: "UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ADMITTED", + 2: "UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ACCEPTED", + 3: "UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED", + } + UpdateWorkflowExecutionLifecycleStage_value = map[string]int32{ + "UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED": 0, + "UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ADMITTED": 1, + "UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ACCEPTED": 2, + "UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED": 3, + } +) + +func (x UpdateWorkflowExecutionLifecycleStage) Enum() *UpdateWorkflowExecutionLifecycleStage { + p := new(UpdateWorkflowExecutionLifecycleStage) + *p = x + return p +} + +func (x UpdateWorkflowExecutionLifecycleStage) String() string { + switch x { + case UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED: + return "Unspecified" + case UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ADMITTED: + return "Admitted" + case UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ACCEPTED: + return "Accepted" + case UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED: + return "Completed" + default: + return strconv.Itoa(int(x)) + } + +} + +func (UpdateWorkflowExecutionLifecycleStage) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_update_proto_enumTypes[0].Descriptor() +} + +func (UpdateWorkflowExecutionLifecycleStage) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_update_proto_enumTypes[0] +} + +func (x UpdateWorkflowExecutionLifecycleStage) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use UpdateWorkflowExecutionLifecycleStage.Descriptor instead. +func (UpdateWorkflowExecutionLifecycleStage) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_update_proto_rawDescGZIP(), []int{0} +} + +// Records why a WorkflowExecutionUpdateAdmittedEvent was written to history. +// Note that not all admitted updates result in this event. +type UpdateAdmittedEventOrigin int32 + +const ( + UPDATE_ADMITTED_EVENT_ORIGIN_UNSPECIFIED UpdateAdmittedEventOrigin = 0 + // The UpdateAdmitted event was created when reapplying events during reset + // or replication. I.e. an accepted update on one branch of workflow history + // was converted into an admitted update on a different branch. + UPDATE_ADMITTED_EVENT_ORIGIN_REAPPLY UpdateAdmittedEventOrigin = 1 +) + +// Enum value maps for UpdateAdmittedEventOrigin. +var ( + UpdateAdmittedEventOrigin_name = map[int32]string{ + 0: "UPDATE_ADMITTED_EVENT_ORIGIN_UNSPECIFIED", + 1: "UPDATE_ADMITTED_EVENT_ORIGIN_REAPPLY", + } + UpdateAdmittedEventOrigin_value = map[string]int32{ + "UPDATE_ADMITTED_EVENT_ORIGIN_UNSPECIFIED": 0, + "UPDATE_ADMITTED_EVENT_ORIGIN_REAPPLY": 1, + } +) + +func (x UpdateAdmittedEventOrigin) Enum() *UpdateAdmittedEventOrigin { + p := new(UpdateAdmittedEventOrigin) + *p = x + return p +} + +func (x UpdateAdmittedEventOrigin) String() string { + switch x { + case UPDATE_ADMITTED_EVENT_ORIGIN_UNSPECIFIED: + return "Unspecified" + case UPDATE_ADMITTED_EVENT_ORIGIN_REAPPLY: + return "Reapply" + default: + return strconv.Itoa(int(x)) + } + +} + +func (UpdateAdmittedEventOrigin) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_update_proto_enumTypes[1].Descriptor() +} + +func (UpdateAdmittedEventOrigin) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_update_proto_enumTypes[1] +} + +func (x UpdateAdmittedEventOrigin) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use UpdateAdmittedEventOrigin.Descriptor instead. +func (UpdateAdmittedEventOrigin) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_update_proto_rawDescGZIP(), []int{1} +} + +var File_temporal_api_enums_v1_update_proto protoreflect.FileDescriptor + +var file_temporal_api_enums_v1_update_proto_rawDesc = []byte{ + 0x0a, 0x22, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2a, 0x8b, 0x02, 0x0a, 0x25, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x66, 0x65, 0x63, 0x79, 0x63, 0x6c, 0x65, + 0x53, 0x74, 0x61, 0x67, 0x65, 0x12, 0x39, 0x0a, 0x35, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, + 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x4c, 0x49, 0x46, 0x45, 0x43, 0x59, 0x43, 0x4c, 0x45, 0x5f, 0x53, 0x54, 0x41, + 0x47, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, + 0x12, 0x36, 0x0a, 0x32, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4c, 0x49, + 0x46, 0x45, 0x43, 0x59, 0x43, 0x4c, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x47, 0x45, 0x5f, 0x41, 0x44, + 0x4d, 0x49, 0x54, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x36, 0x0a, 0x32, 0x55, 0x50, 0x44, 0x41, + 0x54, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, + 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4c, 0x49, 0x46, 0x45, 0x43, 0x59, 0x43, 0x4c, 0x45, 0x5f, + 0x53, 0x54, 0x41, 0x47, 0x45, 0x5f, 0x41, 0x43, 0x43, 0x45, 0x50, 0x54, 0x45, 0x44, 0x10, 0x02, + 0x12, 0x37, 0x0a, 0x33, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4c, 0x49, + 0x46, 0x45, 0x43, 0x59, 0x43, 0x4c, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x47, 0x45, 0x5f, 0x43, 0x4f, + 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x03, 0x2a, 0x73, 0x0a, 0x19, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x12, 0x2c, 0x0a, 0x28, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, + 0x5f, 0x41, 0x44, 0x4d, 0x49, 0x54, 0x54, 0x45, 0x44, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, + 0x4f, 0x52, 0x49, 0x47, 0x49, 0x4e, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, + 0x45, 0x44, 0x10, 0x00, 0x12, 0x28, 0x0a, 0x24, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, 0x41, + 0x44, 0x4d, 0x49, 0x54, 0x54, 0x45, 0x44, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x4f, 0x52, + 0x49, 0x47, 0x49, 0x4e, 0x5f, 0x52, 0x45, 0x41, 0x50, 0x50, 0x4c, 0x59, 0x10, 0x01, 0x42, 0x83, + 0x01, 0x0a, 0x18, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x42, 0x0b, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x21, 0x67, 0x6f, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x3b, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0xaa, 0x02, 0x17, + 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x45, + 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1a, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x45, 0x6e, 0x75, 0x6d, 0x73, + 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_enums_v1_update_proto_rawDescOnce sync.Once + file_temporal_api_enums_v1_update_proto_rawDescData = file_temporal_api_enums_v1_update_proto_rawDesc +) + +func file_temporal_api_enums_v1_update_proto_rawDescGZIP() []byte { + file_temporal_api_enums_v1_update_proto_rawDescOnce.Do(func() { + file_temporal_api_enums_v1_update_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_enums_v1_update_proto_rawDescData) + }) + return file_temporal_api_enums_v1_update_proto_rawDescData +} + +var file_temporal_api_enums_v1_update_proto_enumTypes = make([]protoimpl.EnumInfo, 2) +var file_temporal_api_enums_v1_update_proto_goTypes = []any{ + (UpdateWorkflowExecutionLifecycleStage)(0), // 0: temporal.api.enums.v1.UpdateWorkflowExecutionLifecycleStage + (UpdateAdmittedEventOrigin)(0), // 1: temporal.api.enums.v1.UpdateAdmittedEventOrigin +} +var file_temporal_api_enums_v1_update_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_enums_v1_update_proto_init() } +func file_temporal_api_enums_v1_update_proto_init() { + if File_temporal_api_enums_v1_update_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_enums_v1_update_proto_rawDesc, + NumEnums: 2, + NumMessages: 0, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_enums_v1_update_proto_goTypes, + DependencyIndexes: file_temporal_api_enums_v1_update_proto_depIdxs, + EnumInfos: file_temporal_api_enums_v1_update_proto_enumTypes, + }.Build() + File_temporal_api_enums_v1_update_proto = out.File + file_temporal_api_enums_v1_update_proto_rawDesc = nil + file_temporal_api_enums_v1_update_proto_goTypes = nil + file_temporal_api_enums_v1_update_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/enums/v1/workflow.go-helpers.pb.go b/vendor/go.temporal.io/api/enums/v1/workflow.go-helpers.pb.go new file mode 100644 index 00000000000..8de1e3b65af --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/workflow.go-helpers.pb.go @@ -0,0 +1,236 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package enums + +import ( + "fmt" +) + +var ( + WorkflowIdReusePolicy_shorthandValue = map[string]int32{ + "Unspecified": 0, + "AllowDuplicate": 1, + "AllowDuplicateFailedOnly": 2, + "RejectDuplicate": 3, + "TerminateIfRunning": 4, + } +) + +// WorkflowIdReusePolicyFromString parses a WorkflowIdReusePolicy value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to WorkflowIdReusePolicy +func WorkflowIdReusePolicyFromString(s string) (WorkflowIdReusePolicy, error) { + if v, ok := WorkflowIdReusePolicy_value[s]; ok { + return WorkflowIdReusePolicy(v), nil + } else if v, ok := WorkflowIdReusePolicy_shorthandValue[s]; ok { + return WorkflowIdReusePolicy(v), nil + } + return WorkflowIdReusePolicy(0), fmt.Errorf("%s is not a valid WorkflowIdReusePolicy", s) +} + +var ( + WorkflowIdConflictPolicy_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Fail": 1, + "UseExisting": 2, + "TerminateExisting": 3, + } +) + +// WorkflowIdConflictPolicyFromString parses a WorkflowIdConflictPolicy value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to WorkflowIdConflictPolicy +func WorkflowIdConflictPolicyFromString(s string) (WorkflowIdConflictPolicy, error) { + if v, ok := WorkflowIdConflictPolicy_value[s]; ok { + return WorkflowIdConflictPolicy(v), nil + } else if v, ok := WorkflowIdConflictPolicy_shorthandValue[s]; ok { + return WorkflowIdConflictPolicy(v), nil + } + return WorkflowIdConflictPolicy(0), fmt.Errorf("%s is not a valid WorkflowIdConflictPolicy", s) +} + +var ( + ParentClosePolicy_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Terminate": 1, + "Abandon": 2, + "RequestCancel": 3, + } +) + +// ParentClosePolicyFromString parses a ParentClosePolicy value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to ParentClosePolicy +func ParentClosePolicyFromString(s string) (ParentClosePolicy, error) { + if v, ok := ParentClosePolicy_value[s]; ok { + return ParentClosePolicy(v), nil + } else if v, ok := ParentClosePolicy_shorthandValue[s]; ok { + return ParentClosePolicy(v), nil + } + return ParentClosePolicy(0), fmt.Errorf("%s is not a valid ParentClosePolicy", s) +} + +var ( + ContinueAsNewInitiator_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Workflow": 1, + "Retry": 2, + "CronSchedule": 3, + } +) + +// ContinueAsNewInitiatorFromString parses a ContinueAsNewInitiator value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to ContinueAsNewInitiator +func ContinueAsNewInitiatorFromString(s string) (ContinueAsNewInitiator, error) { + if v, ok := ContinueAsNewInitiator_value[s]; ok { + return ContinueAsNewInitiator(v), nil + } else if v, ok := ContinueAsNewInitiator_shorthandValue[s]; ok { + return ContinueAsNewInitiator(v), nil + } + return ContinueAsNewInitiator(0), fmt.Errorf("%s is not a valid ContinueAsNewInitiator", s) +} + +var ( + WorkflowExecutionStatus_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Running": 1, + "Completed": 2, + "Failed": 3, + "Canceled": 4, + "Terminated": 5, + "ContinuedAsNew": 6, + "TimedOut": 7, + } +) + +// WorkflowExecutionStatusFromString parses a WorkflowExecutionStatus value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to WorkflowExecutionStatus +func WorkflowExecutionStatusFromString(s string) (WorkflowExecutionStatus, error) { + if v, ok := WorkflowExecutionStatus_value[s]; ok { + return WorkflowExecutionStatus(v), nil + } else if v, ok := WorkflowExecutionStatus_shorthandValue[s]; ok { + return WorkflowExecutionStatus(v), nil + } + return WorkflowExecutionStatus(0), fmt.Errorf("%s is not a valid WorkflowExecutionStatus", s) +} + +var ( + PendingActivityState_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Scheduled": 1, + "Started": 2, + "CancelRequested": 3, + } +) + +// PendingActivityStateFromString parses a PendingActivityState value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to PendingActivityState +func PendingActivityStateFromString(s string) (PendingActivityState, error) { + if v, ok := PendingActivityState_value[s]; ok { + return PendingActivityState(v), nil + } else if v, ok := PendingActivityState_shorthandValue[s]; ok { + return PendingActivityState(v), nil + } + return PendingActivityState(0), fmt.Errorf("%s is not a valid PendingActivityState", s) +} + +var ( + PendingWorkflowTaskState_shorthandValue = map[string]int32{ + "Unspecified": 0, + "Scheduled": 1, + "Started": 2, + } +) + +// PendingWorkflowTaskStateFromString parses a PendingWorkflowTaskState value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to PendingWorkflowTaskState +func PendingWorkflowTaskStateFromString(s string) (PendingWorkflowTaskState, error) { + if v, ok := PendingWorkflowTaskState_value[s]; ok { + return PendingWorkflowTaskState(v), nil + } else if v, ok := PendingWorkflowTaskState_shorthandValue[s]; ok { + return PendingWorkflowTaskState(v), nil + } + return PendingWorkflowTaskState(0), fmt.Errorf("%s is not a valid PendingWorkflowTaskState", s) +} + +var ( + HistoryEventFilterType_shorthandValue = map[string]int32{ + "Unspecified": 0, + "AllEvent": 1, + "CloseEvent": 2, + } +) + +// HistoryEventFilterTypeFromString parses a HistoryEventFilterType value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to HistoryEventFilterType +func HistoryEventFilterTypeFromString(s string) (HistoryEventFilterType, error) { + if v, ok := HistoryEventFilterType_value[s]; ok { + return HistoryEventFilterType(v), nil + } else if v, ok := HistoryEventFilterType_shorthandValue[s]; ok { + return HistoryEventFilterType(v), nil + } + return HistoryEventFilterType(0), fmt.Errorf("%s is not a valid HistoryEventFilterType", s) +} + +var ( + RetryState_shorthandValue = map[string]int32{ + "Unspecified": 0, + "InProgress": 1, + "NonRetryableFailure": 2, + "Timeout": 3, + "MaximumAttemptsReached": 4, + "RetryPolicyNotSet": 5, + "InternalServerError": 6, + "CancelRequested": 7, + } +) + +// RetryStateFromString parses a RetryState value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to RetryState +func RetryStateFromString(s string) (RetryState, error) { + if v, ok := RetryState_value[s]; ok { + return RetryState(v), nil + } else if v, ok := RetryState_shorthandValue[s]; ok { + return RetryState(v), nil + } + return RetryState(0), fmt.Errorf("%s is not a valid RetryState", s) +} + +var ( + TimeoutType_shorthandValue = map[string]int32{ + "Unspecified": 0, + "StartToClose": 1, + "ScheduleToStart": 2, + "ScheduleToClose": 3, + "Heartbeat": 4, + } +) + +// TimeoutTypeFromString parses a TimeoutType value from either the protojson +// canonical SCREAMING_CASE enum or the traditional temporal PascalCase enum to TimeoutType +func TimeoutTypeFromString(s string) (TimeoutType, error) { + if v, ok := TimeoutType_value[s]; ok { + return TimeoutType(v), nil + } else if v, ok := TimeoutType_shorthandValue[s]; ok { + return TimeoutType(v), nil + } + return TimeoutType(0), fmt.Errorf("%s is not a valid TimeoutType", s) +} diff --git a/vendor/go.temporal.io/api/enums/v1/workflow.pb.go b/vendor/go.temporal.io/api/enums/v1/workflow.pb.go new file mode 100644 index 00000000000..6544c3397e9 --- /dev/null +++ b/vendor/go.temporal.io/api/enums/v1/workflow.pb.go @@ -0,0 +1,971 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/enums/v1/workflow.proto + +package enums + +import ( + reflect "reflect" + "strconv" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// Defines whether to allow re-using a workflow id from a previously *closed* workflow. +// If the request is denied, a `WorkflowExecutionAlreadyStartedFailure` is returned. +// +// See `WorkflowIdConflictPolicy` for handling workflow id duplication with a *running* workflow. +type WorkflowIdReusePolicy int32 + +const ( + WORKFLOW_ID_REUSE_POLICY_UNSPECIFIED WorkflowIdReusePolicy = 0 + // Allow starting a workflow execution using the same workflow id. + WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE WorkflowIdReusePolicy = 1 + // Allow starting a workflow execution using the same workflow id, only when the last + // execution's final state is one of [terminated, cancelled, timed out, failed]. + WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY WorkflowIdReusePolicy = 2 + // Do not permit re-use of the workflow id for this workflow. Future start workflow requests + // could potentially change the policy, allowing re-use of the workflow id. + WORKFLOW_ID_REUSE_POLICY_REJECT_DUPLICATE WorkflowIdReusePolicy = 3 + // This option belongs in WorkflowIdConflictPolicy but is here for backwards compatibility. + // If specified, it acts like ALLOW_DUPLICATE, but also the WorkflowId*Conflict*Policy on + // the request is treated as WORKFLOW_ID_CONFLICT_POLICY_TERMINATE_EXISTING. + // If no running workflow, then the behavior is the same as ALLOW_DUPLICATE. + WORKFLOW_ID_REUSE_POLICY_TERMINATE_IF_RUNNING WorkflowIdReusePolicy = 4 +) + +// Enum value maps for WorkflowIdReusePolicy. +var ( + WorkflowIdReusePolicy_name = map[int32]string{ + 0: "WORKFLOW_ID_REUSE_POLICY_UNSPECIFIED", + 1: "WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE", + 2: "WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY", + 3: "WORKFLOW_ID_REUSE_POLICY_REJECT_DUPLICATE", + 4: "WORKFLOW_ID_REUSE_POLICY_TERMINATE_IF_RUNNING", + } + WorkflowIdReusePolicy_value = map[string]int32{ + "WORKFLOW_ID_REUSE_POLICY_UNSPECIFIED": 0, + "WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE": 1, + "WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY": 2, + "WORKFLOW_ID_REUSE_POLICY_REJECT_DUPLICATE": 3, + "WORKFLOW_ID_REUSE_POLICY_TERMINATE_IF_RUNNING": 4, + } +) + +func (x WorkflowIdReusePolicy) Enum() *WorkflowIdReusePolicy { + p := new(WorkflowIdReusePolicy) + *p = x + return p +} + +func (x WorkflowIdReusePolicy) String() string { + switch x { + case WORKFLOW_ID_REUSE_POLICY_UNSPECIFIED: + return "Unspecified" + case WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE: + return "AllowDuplicate" + case WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY: + return "AllowDuplicateFailedOnly" + case WORKFLOW_ID_REUSE_POLICY_REJECT_DUPLICATE: + return "RejectDuplicate" + case WORKFLOW_ID_REUSE_POLICY_TERMINATE_IF_RUNNING: + return "TerminateIfRunning" + default: + return strconv.Itoa(int(x)) + } + +} + +func (WorkflowIdReusePolicy) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_workflow_proto_enumTypes[0].Descriptor() +} + +func (WorkflowIdReusePolicy) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_workflow_proto_enumTypes[0] +} + +func (x WorkflowIdReusePolicy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use WorkflowIdReusePolicy.Descriptor instead. +func (WorkflowIdReusePolicy) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_workflow_proto_rawDescGZIP(), []int{0} +} + +// Defines what to do when trying to start a workflow with the same workflow id as a *running* workflow. +// Note that it is *never* valid to have two actively running instances of the same workflow id. +// +// See `WorkflowIdReusePolicy` for handling workflow id duplication with a *closed* workflow. +type WorkflowIdConflictPolicy int32 + +const ( + WORKFLOW_ID_CONFLICT_POLICY_UNSPECIFIED WorkflowIdConflictPolicy = 0 + // Don't start a new workflow; instead return `WorkflowExecutionAlreadyStartedFailure`. + WORKFLOW_ID_CONFLICT_POLICY_FAIL WorkflowIdConflictPolicy = 1 + // Don't start a new workflow; instead return a workflow handle for the running workflow. + WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING WorkflowIdConflictPolicy = 2 + // Terminate the running workflow before starting a new one. + WORKFLOW_ID_CONFLICT_POLICY_TERMINATE_EXISTING WorkflowIdConflictPolicy = 3 +) + +// Enum value maps for WorkflowIdConflictPolicy. +var ( + WorkflowIdConflictPolicy_name = map[int32]string{ + 0: "WORKFLOW_ID_CONFLICT_POLICY_UNSPECIFIED", + 1: "WORKFLOW_ID_CONFLICT_POLICY_FAIL", + 2: "WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING", + 3: "WORKFLOW_ID_CONFLICT_POLICY_TERMINATE_EXISTING", + } + WorkflowIdConflictPolicy_value = map[string]int32{ + "WORKFLOW_ID_CONFLICT_POLICY_UNSPECIFIED": 0, + "WORKFLOW_ID_CONFLICT_POLICY_FAIL": 1, + "WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING": 2, + "WORKFLOW_ID_CONFLICT_POLICY_TERMINATE_EXISTING": 3, + } +) + +func (x WorkflowIdConflictPolicy) Enum() *WorkflowIdConflictPolicy { + p := new(WorkflowIdConflictPolicy) + *p = x + return p +} + +func (x WorkflowIdConflictPolicy) String() string { + switch x { + case WORKFLOW_ID_CONFLICT_POLICY_UNSPECIFIED: + return "Unspecified" + case WORKFLOW_ID_CONFLICT_POLICY_FAIL: + return "Fail" + case WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING: + return "UseExisting" + case WORKFLOW_ID_CONFLICT_POLICY_TERMINATE_EXISTING: + return "TerminateExisting" + default: + return strconv.Itoa(int(x)) + } + +} + +func (WorkflowIdConflictPolicy) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_workflow_proto_enumTypes[1].Descriptor() +} + +func (WorkflowIdConflictPolicy) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_workflow_proto_enumTypes[1] +} + +func (x WorkflowIdConflictPolicy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use WorkflowIdConflictPolicy.Descriptor instead. +func (WorkflowIdConflictPolicy) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_workflow_proto_rawDescGZIP(), []int{1} +} + +// Defines how child workflows will react to their parent completing +type ParentClosePolicy int32 + +const ( + PARENT_CLOSE_POLICY_UNSPECIFIED ParentClosePolicy = 0 + // The child workflow will also terminate + PARENT_CLOSE_POLICY_TERMINATE ParentClosePolicy = 1 + // The child workflow will do nothing + PARENT_CLOSE_POLICY_ABANDON ParentClosePolicy = 2 + // Cancellation will be requested of the child workflow + PARENT_CLOSE_POLICY_REQUEST_CANCEL ParentClosePolicy = 3 +) + +// Enum value maps for ParentClosePolicy. +var ( + ParentClosePolicy_name = map[int32]string{ + 0: "PARENT_CLOSE_POLICY_UNSPECIFIED", + 1: "PARENT_CLOSE_POLICY_TERMINATE", + 2: "PARENT_CLOSE_POLICY_ABANDON", + 3: "PARENT_CLOSE_POLICY_REQUEST_CANCEL", + } + ParentClosePolicy_value = map[string]int32{ + "PARENT_CLOSE_POLICY_UNSPECIFIED": 0, + "PARENT_CLOSE_POLICY_TERMINATE": 1, + "PARENT_CLOSE_POLICY_ABANDON": 2, + "PARENT_CLOSE_POLICY_REQUEST_CANCEL": 3, + } +) + +func (x ParentClosePolicy) Enum() *ParentClosePolicy { + p := new(ParentClosePolicy) + *p = x + return p +} + +func (x ParentClosePolicy) String() string { + switch x { + case PARENT_CLOSE_POLICY_UNSPECIFIED: + return "Unspecified" + case PARENT_CLOSE_POLICY_TERMINATE: + return "Terminate" + case PARENT_CLOSE_POLICY_ABANDON: + return "Abandon" + case PARENT_CLOSE_POLICY_REQUEST_CANCEL: + return "RequestCancel" + default: + return strconv.Itoa(int(x)) + } + +} + +func (ParentClosePolicy) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_workflow_proto_enumTypes[2].Descriptor() +} + +func (ParentClosePolicy) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_workflow_proto_enumTypes[2] +} + +func (x ParentClosePolicy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ParentClosePolicy.Descriptor instead. +func (ParentClosePolicy) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_workflow_proto_rawDescGZIP(), []int{2} +} + +type ContinueAsNewInitiator int32 + +const ( + CONTINUE_AS_NEW_INITIATOR_UNSPECIFIED ContinueAsNewInitiator = 0 + // The workflow itself requested to continue as new + CONTINUE_AS_NEW_INITIATOR_WORKFLOW ContinueAsNewInitiator = 1 + // The workflow continued as new because it is retrying + CONTINUE_AS_NEW_INITIATOR_RETRY ContinueAsNewInitiator = 2 + // The workflow continued as new because cron has triggered a new execution + CONTINUE_AS_NEW_INITIATOR_CRON_SCHEDULE ContinueAsNewInitiator = 3 +) + +// Enum value maps for ContinueAsNewInitiator. +var ( + ContinueAsNewInitiator_name = map[int32]string{ + 0: "CONTINUE_AS_NEW_INITIATOR_UNSPECIFIED", + 1: "CONTINUE_AS_NEW_INITIATOR_WORKFLOW", + 2: "CONTINUE_AS_NEW_INITIATOR_RETRY", + 3: "CONTINUE_AS_NEW_INITIATOR_CRON_SCHEDULE", + } + ContinueAsNewInitiator_value = map[string]int32{ + "CONTINUE_AS_NEW_INITIATOR_UNSPECIFIED": 0, + "CONTINUE_AS_NEW_INITIATOR_WORKFLOW": 1, + "CONTINUE_AS_NEW_INITIATOR_RETRY": 2, + "CONTINUE_AS_NEW_INITIATOR_CRON_SCHEDULE": 3, + } +) + +func (x ContinueAsNewInitiator) Enum() *ContinueAsNewInitiator { + p := new(ContinueAsNewInitiator) + *p = x + return p +} + +func (x ContinueAsNewInitiator) String() string { + switch x { + case CONTINUE_AS_NEW_INITIATOR_UNSPECIFIED: + return "Unspecified" + case CONTINUE_AS_NEW_INITIATOR_WORKFLOW: + return "Workflow" + case CONTINUE_AS_NEW_INITIATOR_RETRY: + return "Retry" + case CONTINUE_AS_NEW_INITIATOR_CRON_SCHEDULE: + return "CronSchedule" + default: + return strconv.Itoa(int(x)) + } + +} + +func (ContinueAsNewInitiator) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_workflow_proto_enumTypes[3].Descriptor() +} + +func (ContinueAsNewInitiator) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_workflow_proto_enumTypes[3] +} + +func (x ContinueAsNewInitiator) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ContinueAsNewInitiator.Descriptor instead. +func (ContinueAsNewInitiator) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_workflow_proto_rawDescGZIP(), []int{3} +} + +// (-- api-linter: core::0216::synonyms=disabled +// +// aip.dev/not-precedent: There is WorkflowExecutionState already in another package. --) +type WorkflowExecutionStatus int32 + +const ( + WORKFLOW_EXECUTION_STATUS_UNSPECIFIED WorkflowExecutionStatus = 0 + // Value 1 is hardcoded in SQL persistence. + WORKFLOW_EXECUTION_STATUS_RUNNING WorkflowExecutionStatus = 1 + WORKFLOW_EXECUTION_STATUS_COMPLETED WorkflowExecutionStatus = 2 + WORKFLOW_EXECUTION_STATUS_FAILED WorkflowExecutionStatus = 3 + WORKFLOW_EXECUTION_STATUS_CANCELED WorkflowExecutionStatus = 4 + WORKFLOW_EXECUTION_STATUS_TERMINATED WorkflowExecutionStatus = 5 + WORKFLOW_EXECUTION_STATUS_CONTINUED_AS_NEW WorkflowExecutionStatus = 6 + WORKFLOW_EXECUTION_STATUS_TIMED_OUT WorkflowExecutionStatus = 7 +) + +// Enum value maps for WorkflowExecutionStatus. +var ( + WorkflowExecutionStatus_name = map[int32]string{ + 0: "WORKFLOW_EXECUTION_STATUS_UNSPECIFIED", + 1: "WORKFLOW_EXECUTION_STATUS_RUNNING", + 2: "WORKFLOW_EXECUTION_STATUS_COMPLETED", + 3: "WORKFLOW_EXECUTION_STATUS_FAILED", + 4: "WORKFLOW_EXECUTION_STATUS_CANCELED", + 5: "WORKFLOW_EXECUTION_STATUS_TERMINATED", + 6: "WORKFLOW_EXECUTION_STATUS_CONTINUED_AS_NEW", + 7: "WORKFLOW_EXECUTION_STATUS_TIMED_OUT", + } + WorkflowExecutionStatus_value = map[string]int32{ + "WORKFLOW_EXECUTION_STATUS_UNSPECIFIED": 0, + "WORKFLOW_EXECUTION_STATUS_RUNNING": 1, + "WORKFLOW_EXECUTION_STATUS_COMPLETED": 2, + "WORKFLOW_EXECUTION_STATUS_FAILED": 3, + "WORKFLOW_EXECUTION_STATUS_CANCELED": 4, + "WORKFLOW_EXECUTION_STATUS_TERMINATED": 5, + "WORKFLOW_EXECUTION_STATUS_CONTINUED_AS_NEW": 6, + "WORKFLOW_EXECUTION_STATUS_TIMED_OUT": 7, + } +) + +func (x WorkflowExecutionStatus) Enum() *WorkflowExecutionStatus { + p := new(WorkflowExecutionStatus) + *p = x + return p +} + +func (x WorkflowExecutionStatus) String() string { + switch x { + case WORKFLOW_EXECUTION_STATUS_UNSPECIFIED: + return "Unspecified" + case WORKFLOW_EXECUTION_STATUS_RUNNING: + return "Running" + case WORKFLOW_EXECUTION_STATUS_COMPLETED: + return "Completed" + case WORKFLOW_EXECUTION_STATUS_FAILED: + return "Failed" + case WORKFLOW_EXECUTION_STATUS_CANCELED: + return "Canceled" + case WORKFLOW_EXECUTION_STATUS_TERMINATED: + return "Terminated" + case WORKFLOW_EXECUTION_STATUS_CONTINUED_AS_NEW: + return "ContinuedAsNew" + case WORKFLOW_EXECUTION_STATUS_TIMED_OUT: + return "TimedOut" + default + + // Deprecated: Use WorkflowExecutionStatus.Descriptor instead. + : + return strconv.Itoa(int(x)) + } + +} + +func (WorkflowExecutionStatus) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_workflow_proto_enumTypes[4].Descriptor() +} + +func (WorkflowExecutionStatus) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_workflow_proto_enumTypes[4] +} + +func (x WorkflowExecutionStatus) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +func (WorkflowExecutionStatus) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_workflow_proto_rawDescGZIP(), []int{4} +} + +type PendingActivityState int32 + +const ( + PENDING_ACTIVITY_STATE_UNSPECIFIED PendingActivityState = 0 + PENDING_ACTIVITY_STATE_SCHEDULED PendingActivityState = 1 + PENDING_ACTIVITY_STATE_STARTED PendingActivityState = 2 + PENDING_ACTIVITY_STATE_CANCEL_REQUESTED PendingActivityState = 3 +) + +// Enum value maps for PendingActivityState. +var ( + PendingActivityState_name = map[int32]string{ + 0: "PENDING_ACTIVITY_STATE_UNSPECIFIED", + 1: "PENDING_ACTIVITY_STATE_SCHEDULED", + 2: "PENDING_ACTIVITY_STATE_STARTED", + 3: "PENDING_ACTIVITY_STATE_CANCEL_REQUESTED", + } + PendingActivityState_value = map[string]int32{ + "PENDING_ACTIVITY_STATE_UNSPECIFIED": 0, + "PENDING_ACTIVITY_STATE_SCHEDULED": 1, + "PENDING_ACTIVITY_STATE_STARTED": 2, + "PENDING_ACTIVITY_STATE_CANCEL_REQUESTED": 3, + } +) + +func (x PendingActivityState) Enum() *PendingActivityState { + p := new(PendingActivityState) + *p = x + return p +} + +func (x PendingActivityState) String() string { + switch x { + case PENDING_ACTIVITY_STATE_UNSPECIFIED: + return "Unspecified" + case PENDING_ACTIVITY_STATE_SCHEDULED: + return "Scheduled" + case PENDING_ACTIVITY_STATE_STARTED: + return "Started" + case PENDING_ACTIVITY_STATE_CANCEL_REQUESTED: + return "CancelRequested" + default: + return strconv.Itoa(int(x)) + } + +} + +func (PendingActivityState) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_workflow_proto_enumTypes[5].Descriptor() +} + +func (PendingActivityState) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_workflow_proto_enumTypes[5] +} + +func (x PendingActivityState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use PendingActivityState.Descriptor instead. +func (PendingActivityState) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_workflow_proto_rawDescGZIP(), []int{5} +} + +type PendingWorkflowTaskState int32 + +const ( + PENDING_WORKFLOW_TASK_STATE_UNSPECIFIED PendingWorkflowTaskState = 0 + PENDING_WORKFLOW_TASK_STATE_SCHEDULED PendingWorkflowTaskState = 1 + PENDING_WORKFLOW_TASK_STATE_STARTED PendingWorkflowTaskState = 2 +) + +// Enum value maps for PendingWorkflowTaskState. +var ( + PendingWorkflowTaskState_name = map[int32]string{ + 0: "PENDING_WORKFLOW_TASK_STATE_UNSPECIFIED", + 1: "PENDING_WORKFLOW_TASK_STATE_SCHEDULED", + 2: "PENDING_WORKFLOW_TASK_STATE_STARTED", + } + PendingWorkflowTaskState_value = map[string]int32{ + "PENDING_WORKFLOW_TASK_STATE_UNSPECIFIED": 0, + "PENDING_WORKFLOW_TASK_STATE_SCHEDULED": 1, + "PENDING_WORKFLOW_TASK_STATE_STARTED": 2, + } +) + +func (x PendingWorkflowTaskState) Enum() *PendingWorkflowTaskState { + p := new(PendingWorkflowTaskState) + *p = x + return p +} + +func (x PendingWorkflowTaskState) String() string { + switch x { + case PENDING_WORKFLOW_TASK_STATE_UNSPECIFIED: + return "Unspecified" + case PENDING_WORKFLOW_TASK_STATE_SCHEDULED: + return "Scheduled" + case PENDING_WORKFLOW_TASK_STATE_STARTED: + return "Started" + default: + return strconv.Itoa(int(x)) + } + +} + +func (PendingWorkflowTaskState) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_workflow_proto_enumTypes[6].Descriptor() +} + +func (PendingWorkflowTaskState) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_workflow_proto_enumTypes[6] +} + +func (x PendingWorkflowTaskState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use PendingWorkflowTaskState.Descriptor instead. +func (PendingWorkflowTaskState) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_workflow_proto_rawDescGZIP(), []int{6} +} + +type HistoryEventFilterType int32 + +const ( + HISTORY_EVENT_FILTER_TYPE_UNSPECIFIED HistoryEventFilterType = 0 + HISTORY_EVENT_FILTER_TYPE_ALL_EVENT HistoryEventFilterType = 1 + HISTORY_EVENT_FILTER_TYPE_CLOSE_EVENT HistoryEventFilterType = 2 +) + +// Enum value maps for HistoryEventFilterType. +var ( + HistoryEventFilterType_name = map[int32]string{ + 0: "HISTORY_EVENT_FILTER_TYPE_UNSPECIFIED", + 1: "HISTORY_EVENT_FILTER_TYPE_ALL_EVENT", + 2: "HISTORY_EVENT_FILTER_TYPE_CLOSE_EVENT", + } + HistoryEventFilterType_value = map[string]int32{ + "HISTORY_EVENT_FILTER_TYPE_UNSPECIFIED": 0, + "HISTORY_EVENT_FILTER_TYPE_ALL_EVENT": 1, + "HISTORY_EVENT_FILTER_TYPE_CLOSE_EVENT": 2, + } +) + +func (x HistoryEventFilterType) Enum() *HistoryEventFilterType { + p := new(HistoryEventFilterType) + *p = x + return p +} + +func (x HistoryEventFilterType) String() string { + switch x { + case HISTORY_EVENT_FILTER_TYPE_UNSPECIFIED: + return "Unspecified" + case HISTORY_EVENT_FILTER_TYPE_ALL_EVENT: + return "AllEvent" + case HISTORY_EVENT_FILTER_TYPE_CLOSE_EVENT: + return "CloseEvent" + default: + return strconv.Itoa(int(x)) + } + +} + +func (HistoryEventFilterType) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_workflow_proto_enumTypes[7].Descriptor() +} + +func (HistoryEventFilterType) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_workflow_proto_enumTypes[7] +} + +func (x HistoryEventFilterType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use HistoryEventFilterType.Descriptor instead. +func (HistoryEventFilterType) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_workflow_proto_rawDescGZIP(), []int{7} +} + +type RetryState int32 + +const ( + RETRY_STATE_UNSPECIFIED RetryState = 0 + RETRY_STATE_IN_PROGRESS RetryState = 1 + RETRY_STATE_NON_RETRYABLE_FAILURE RetryState = 2 + RETRY_STATE_TIMEOUT RetryState = 3 + RETRY_STATE_MAXIMUM_ATTEMPTS_REACHED RetryState = 4 + RETRY_STATE_RETRY_POLICY_NOT_SET RetryState = 5 + RETRY_STATE_INTERNAL_SERVER_ERROR RetryState = 6 + RETRY_STATE_CANCEL_REQUESTED RetryState = 7 +) + +// Enum value maps for RetryState. +var ( + RetryState_name = map[int32]string{ + 0: "RETRY_STATE_UNSPECIFIED", + 1: "RETRY_STATE_IN_PROGRESS", + 2: "RETRY_STATE_NON_RETRYABLE_FAILURE", + 3: "RETRY_STATE_TIMEOUT", + 4: "RETRY_STATE_MAXIMUM_ATTEMPTS_REACHED", + 5: "RETRY_STATE_RETRY_POLICY_NOT_SET", + 6: "RETRY_STATE_INTERNAL_SERVER_ERROR", + 7: "RETRY_STATE_CANCEL_REQUESTED", + } + RetryState_value = map[string]int32{ + "RETRY_STATE_UNSPECIFIED": 0, + "RETRY_STATE_IN_PROGRESS": 1, + "RETRY_STATE_NON_RETRYABLE_FAILURE": 2, + "RETRY_STATE_TIMEOUT": 3, + "RETRY_STATE_MAXIMUM_ATTEMPTS_REACHED": 4, + "RETRY_STATE_RETRY_POLICY_NOT_SET": 5, + "RETRY_STATE_INTERNAL_SERVER_ERROR": 6, + "RETRY_STATE_CANCEL_REQUESTED": 7, + } +) + +func (x RetryState) Enum() *RetryState { + p := new(RetryState) + *p = x + return p +} + +func (x RetryState) String() string { + switch x { + case RETRY_STATE_UNSPECIFIED: + return "Unspecified" + case RETRY_STATE_IN_PROGRESS: + return "InProgress" + case RETRY_STATE_NON_RETRYABLE_FAILURE: + return "NonRetryableFailure" + case RETRY_STATE_TIMEOUT: + return "Timeout" + case RETRY_STATE_MAXIMUM_ATTEMPTS_REACHED: + return "MaximumAttemptsReached" + case RETRY_STATE_RETRY_POLICY_NOT_SET: + return "RetryPolicyNotSet" + case RETRY_STATE_INTERNAL_SERVER_ERROR: + return "InternalServerError" + case RETRY_STATE_CANCEL_REQUESTED: + + // Deprecated: Use RetryState.Descriptor instead. + return "CancelRequested" + default: + return strconv.Itoa(int(x)) + } + +} + +func (RetryState) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_workflow_proto_enumTypes[8].Descriptor() +} + +func (RetryState) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_workflow_proto_enumTypes[8] +} + +func (x RetryState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +func (RetryState) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_workflow_proto_rawDescGZIP(), []int{8} +} + +type TimeoutType int32 + +const ( + TIMEOUT_TYPE_UNSPECIFIED TimeoutType = 0 + TIMEOUT_TYPE_START_TO_CLOSE TimeoutType = 1 + TIMEOUT_TYPE_SCHEDULE_TO_START TimeoutType = 2 + TIMEOUT_TYPE_SCHEDULE_TO_CLOSE TimeoutType = 3 + TIMEOUT_TYPE_HEARTBEAT TimeoutType = 4 +) + +// Enum value maps for TimeoutType. +var ( + TimeoutType_name = map[int32]string{ + 0: "TIMEOUT_TYPE_UNSPECIFIED", + 1: "TIMEOUT_TYPE_START_TO_CLOSE", + 2: "TIMEOUT_TYPE_SCHEDULE_TO_START", + 3: "TIMEOUT_TYPE_SCHEDULE_TO_CLOSE", + 4: "TIMEOUT_TYPE_HEARTBEAT", + } + TimeoutType_value = map[string]int32{ + "TIMEOUT_TYPE_UNSPECIFIED": 0, + "TIMEOUT_TYPE_START_TO_CLOSE": 1, + "TIMEOUT_TYPE_SCHEDULE_TO_START": 2, + "TIMEOUT_TYPE_SCHEDULE_TO_CLOSE": 3, + "TIMEOUT_TYPE_HEARTBEAT": 4, + } +) + +func (x TimeoutType) Enum() *TimeoutType { + p := new(TimeoutType) + *p = x + return p +} + +func (x TimeoutType) String() string { + switch x { + case TIMEOUT_TYPE_UNSPECIFIED: + return "Unspecified" + case TIMEOUT_TYPE_START_TO_CLOSE: + return "StartToClose" + case TIMEOUT_TYPE_SCHEDULE_TO_START: + return "ScheduleToStart" + case TIMEOUT_TYPE_SCHEDULE_TO_CLOSE: + return "ScheduleToClose" + case TIMEOUT_TYPE_HEARTBEAT: + return "Heartbeat" + default: + return strconv.Itoa(int(x)) + } + +} + +func (TimeoutType) Descriptor() protoreflect.EnumDescriptor { + return file_temporal_api_enums_v1_workflow_proto_enumTypes[9].Descriptor() +} + +func (TimeoutType) Type() protoreflect.EnumType { + return &file_temporal_api_enums_v1_workflow_proto_enumTypes[9] +} + +func (x TimeoutType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use TimeoutType.Descriptor instead. +func (TimeoutType) EnumDescriptor() ([]byte, []int) { + return file_temporal_api_enums_v1_workflow_proto_rawDescGZIP(), []int{9} +} + +var File_temporal_api_enums_v1_workflow_proto protoreflect.FileDescriptor + +var file_temporal_api_enums_v1_workflow_proto_rawDesc = []byte{ + 0x0a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2a, 0x8b, 0x02, + 0x0a, 0x15, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x52, 0x65, 0x75, 0x73, + 0x65, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x28, 0x0a, 0x24, 0x57, 0x4f, 0x52, 0x4b, 0x46, + 0x4c, 0x4f, 0x57, 0x5f, 0x49, 0x44, 0x5f, 0x52, 0x45, 0x55, 0x53, 0x45, 0x5f, 0x50, 0x4f, 0x4c, + 0x49, 0x43, 0x59, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, + 0x00, 0x12, 0x2c, 0x0a, 0x28, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x49, 0x44, + 0x5f, 0x52, 0x45, 0x55, 0x53, 0x45, 0x5f, 0x50, 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x41, 0x4c, + 0x4c, 0x4f, 0x57, 0x5f, 0x44, 0x55, 0x50, 0x4c, 0x49, 0x43, 0x41, 0x54, 0x45, 0x10, 0x01, 0x12, + 0x38, 0x0a, 0x34, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x49, 0x44, 0x5f, 0x52, + 0x45, 0x55, 0x53, 0x45, 0x5f, 0x50, 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x41, 0x4c, 0x4c, 0x4f, + 0x57, 0x5f, 0x44, 0x55, 0x50, 0x4c, 0x49, 0x43, 0x41, 0x54, 0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, + 0x45, 0x44, 0x5f, 0x4f, 0x4e, 0x4c, 0x59, 0x10, 0x02, 0x12, 0x2d, 0x0a, 0x29, 0x57, 0x4f, 0x52, + 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x49, 0x44, 0x5f, 0x52, 0x45, 0x55, 0x53, 0x45, 0x5f, 0x50, + 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x52, 0x45, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x44, 0x55, 0x50, + 0x4c, 0x49, 0x43, 0x41, 0x54, 0x45, 0x10, 0x03, 0x12, 0x31, 0x0a, 0x2d, 0x57, 0x4f, 0x52, 0x4b, + 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x49, 0x44, 0x5f, 0x52, 0x45, 0x55, 0x53, 0x45, 0x5f, 0x50, 0x4f, + 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, 0x5f, 0x49, + 0x46, 0x5f, 0x52, 0x55, 0x4e, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x04, 0x2a, 0xcf, 0x01, 0x0a, 0x18, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x6c, 0x69, + 0x63, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x2b, 0x0a, 0x27, 0x57, 0x4f, 0x52, 0x4b, + 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x49, 0x44, 0x5f, 0x43, 0x4f, 0x4e, 0x46, 0x4c, 0x49, 0x43, 0x54, + 0x5f, 0x50, 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, + 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, + 0x57, 0x5f, 0x49, 0x44, 0x5f, 0x43, 0x4f, 0x4e, 0x46, 0x4c, 0x49, 0x43, 0x54, 0x5f, 0x50, 0x4f, + 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x10, 0x01, 0x12, 0x2c, 0x0a, 0x28, 0x57, + 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x49, 0x44, 0x5f, 0x43, 0x4f, 0x4e, 0x46, 0x4c, + 0x49, 0x43, 0x54, 0x5f, 0x50, 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x55, 0x53, 0x45, 0x5f, 0x45, + 0x58, 0x49, 0x53, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x32, 0x0a, 0x2e, 0x57, 0x4f, 0x52, + 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x49, 0x44, 0x5f, 0x43, 0x4f, 0x4e, 0x46, 0x4c, 0x49, 0x43, + 0x54, 0x5f, 0x50, 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x49, 0x4e, 0x41, + 0x54, 0x45, 0x5f, 0x45, 0x58, 0x49, 0x53, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x03, 0x2a, 0xa4, 0x01, + 0x0a, 0x11, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x6f, 0x6c, + 0x69, 0x63, 0x79, 0x12, 0x23, 0x0a, 0x1f, 0x50, 0x41, 0x52, 0x45, 0x4e, 0x54, 0x5f, 0x43, 0x4c, + 0x4f, 0x53, 0x45, 0x5f, 0x50, 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, + 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x21, 0x0a, 0x1d, 0x50, 0x41, 0x52, 0x45, + 0x4e, 0x54, 0x5f, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x5f, 0x50, 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, + 0x54, 0x45, 0x52, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, 0x10, 0x01, 0x12, 0x1f, 0x0a, 0x1b, 0x50, + 0x41, 0x52, 0x45, 0x4e, 0x54, 0x5f, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x5f, 0x50, 0x4f, 0x4c, 0x49, + 0x43, 0x59, 0x5f, 0x41, 0x42, 0x41, 0x4e, 0x44, 0x4f, 0x4e, 0x10, 0x02, 0x12, 0x26, 0x0a, 0x22, + 0x50, 0x41, 0x52, 0x45, 0x4e, 0x54, 0x5f, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x5f, 0x50, 0x4f, 0x4c, + 0x49, 0x43, 0x59, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x43, 0x41, 0x4e, 0x43, + 0x45, 0x4c, 0x10, 0x03, 0x2a, 0xbd, 0x01, 0x0a, 0x16, 0x43, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, + 0x65, 0x41, 0x73, 0x4e, 0x65, 0x77, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x6f, 0x72, 0x12, + 0x29, 0x0a, 0x25, 0x43, 0x4f, 0x4e, 0x54, 0x49, 0x4e, 0x55, 0x45, 0x5f, 0x41, 0x53, 0x5f, 0x4e, + 0x45, 0x57, 0x5f, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41, 0x54, 0x4f, 0x52, 0x5f, 0x55, 0x4e, 0x53, + 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x26, 0x0a, 0x22, 0x43, 0x4f, + 0x4e, 0x54, 0x49, 0x4e, 0x55, 0x45, 0x5f, 0x41, 0x53, 0x5f, 0x4e, 0x45, 0x57, 0x5f, 0x49, 0x4e, + 0x49, 0x54, 0x49, 0x41, 0x54, 0x4f, 0x52, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, + 0x10, 0x01, 0x12, 0x23, 0x0a, 0x1f, 0x43, 0x4f, 0x4e, 0x54, 0x49, 0x4e, 0x55, 0x45, 0x5f, 0x41, + 0x53, 0x5f, 0x4e, 0x45, 0x57, 0x5f, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41, 0x54, 0x4f, 0x52, 0x5f, + 0x52, 0x45, 0x54, 0x52, 0x59, 0x10, 0x02, 0x12, 0x2b, 0x0a, 0x27, 0x43, 0x4f, 0x4e, 0x54, 0x49, + 0x4e, 0x55, 0x45, 0x5f, 0x41, 0x53, 0x5f, 0x4e, 0x45, 0x57, 0x5f, 0x49, 0x4e, 0x49, 0x54, 0x49, + 0x41, 0x54, 0x4f, 0x52, 0x5f, 0x43, 0x52, 0x4f, 0x4e, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, + 0x4c, 0x45, 0x10, 0x03, 0x2a, 0xe5, 0x02, 0x0a, 0x17, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x12, 0x29, 0x0a, 0x25, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, + 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x55, 0x4e, + 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x25, 0x0a, 0x21, 0x57, + 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, + 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x52, 0x55, 0x4e, 0x4e, 0x49, 0x4e, 0x47, + 0x10, 0x01, 0x12, 0x27, 0x0a, 0x23, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, + 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, + 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x24, 0x0a, 0x20, 0x57, + 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, + 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, + 0x03, 0x12, 0x26, 0x0a, 0x22, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, + 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x43, + 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x45, 0x44, 0x10, 0x04, 0x12, 0x28, 0x0a, 0x24, 0x57, 0x4f, 0x52, + 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, + 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, + 0x44, 0x10, 0x05, 0x12, 0x2e, 0x0a, 0x2a, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, + 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, + 0x5f, 0x43, 0x4f, 0x4e, 0x54, 0x49, 0x4e, 0x55, 0x45, 0x44, 0x5f, 0x41, 0x53, 0x5f, 0x4e, 0x45, + 0x57, 0x10, 0x06, 0x12, 0x27, 0x0a, 0x23, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, + 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, + 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x07, 0x2a, 0xb5, 0x01, 0x0a, + 0x14, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x26, 0x0a, 0x22, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, + 0x5f, 0x41, 0x43, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, + 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x24, 0x0a, + 0x20, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x56, 0x49, 0x54, + 0x59, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, + 0x44, 0x10, 0x01, 0x12, 0x22, 0x0a, 0x1e, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x41, + 0x43, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x54, + 0x41, 0x52, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x2b, 0x0a, 0x27, 0x50, 0x45, 0x4e, 0x44, 0x49, + 0x4e, 0x47, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x53, 0x54, 0x41, 0x54, + 0x45, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, + 0x45, 0x44, 0x10, 0x03, 0x2a, 0x9b, 0x01, 0x0a, 0x18, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x2b, 0x0a, 0x27, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x57, 0x4f, 0x52, + 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, + 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x29, + 0x0a, 0x25, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, + 0x4f, 0x57, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x43, + 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x44, 0x10, 0x01, 0x12, 0x27, 0x0a, 0x23, 0x50, 0x45, 0x4e, + 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x54, 0x41, + 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, + 0x10, 0x02, 0x2a, 0x97, 0x01, 0x0a, 0x16, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x29, 0x0a, + 0x25, 0x48, 0x49, 0x53, 0x54, 0x4f, 0x52, 0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x46, + 0x49, 0x4c, 0x54, 0x45, 0x52, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, + 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x27, 0x0a, 0x23, 0x48, 0x49, 0x53, 0x54, + 0x4f, 0x52, 0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x46, 0x49, 0x4c, 0x54, 0x45, 0x52, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x41, 0x4c, 0x4c, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x10, + 0x01, 0x12, 0x29, 0x0a, 0x25, 0x48, 0x49, 0x53, 0x54, 0x4f, 0x52, 0x59, 0x5f, 0x45, 0x56, 0x45, + 0x4e, 0x54, 0x5f, 0x46, 0x49, 0x4c, 0x54, 0x45, 0x52, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, + 0x4c, 0x4f, 0x53, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x10, 0x02, 0x2a, 0x9f, 0x02, 0x0a, + 0x0a, 0x52, 0x65, 0x74, 0x72, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1b, 0x0a, 0x17, 0x52, + 0x45, 0x54, 0x52, 0x59, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, + 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x1b, 0x0a, 0x17, 0x52, 0x45, 0x54, 0x52, + 0x59, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x49, 0x4e, 0x5f, 0x50, 0x52, 0x4f, 0x47, 0x52, + 0x45, 0x53, 0x53, 0x10, 0x01, 0x12, 0x25, 0x0a, 0x21, 0x52, 0x45, 0x54, 0x52, 0x59, 0x5f, 0x53, + 0x54, 0x41, 0x54, 0x45, 0x5f, 0x4e, 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x54, 0x52, 0x59, 0x41, 0x42, + 0x4c, 0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x55, 0x52, 0x45, 0x10, 0x02, 0x12, 0x17, 0x0a, 0x13, + 0x52, 0x45, 0x54, 0x52, 0x59, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x54, 0x49, 0x4d, 0x45, + 0x4f, 0x55, 0x54, 0x10, 0x03, 0x12, 0x28, 0x0a, 0x24, 0x52, 0x45, 0x54, 0x52, 0x59, 0x5f, 0x53, + 0x54, 0x41, 0x54, 0x45, 0x5f, 0x4d, 0x41, 0x58, 0x49, 0x4d, 0x55, 0x4d, 0x5f, 0x41, 0x54, 0x54, + 0x45, 0x4d, 0x50, 0x54, 0x53, 0x5f, 0x52, 0x45, 0x41, 0x43, 0x48, 0x45, 0x44, 0x10, 0x04, 0x12, + 0x24, 0x0a, 0x20, 0x52, 0x45, 0x54, 0x52, 0x59, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x52, + 0x45, 0x54, 0x52, 0x59, 0x5f, 0x50, 0x4f, 0x4c, 0x49, 0x43, 0x59, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, + 0x53, 0x45, 0x54, 0x10, 0x05, 0x12, 0x25, 0x0a, 0x21, 0x52, 0x45, 0x54, 0x52, 0x59, 0x5f, 0x53, + 0x54, 0x41, 0x54, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x53, 0x45, + 0x52, 0x56, 0x45, 0x52, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x06, 0x12, 0x20, 0x0a, 0x1c, + 0x52, 0x45, 0x54, 0x52, 0x59, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x43, 0x41, 0x4e, 0x43, + 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x45, 0x44, 0x10, 0x07, 0x2a, 0xb0, + 0x01, 0x0a, 0x0b, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1c, + 0x0a, 0x18, 0x54, 0x49, 0x4d, 0x45, 0x4f, 0x55, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, + 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x1f, 0x0a, 0x1b, + 0x54, 0x49, 0x4d, 0x45, 0x4f, 0x55, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, + 0x52, 0x54, 0x5f, 0x54, 0x4f, 0x5f, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x10, 0x01, 0x12, 0x22, 0x0a, + 0x1e, 0x54, 0x49, 0x4d, 0x45, 0x4f, 0x55, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x43, + 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x5f, 0x54, 0x4f, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x10, + 0x02, 0x12, 0x22, 0x0a, 0x1e, 0x54, 0x49, 0x4d, 0x45, 0x4f, 0x55, 0x54, 0x5f, 0x54, 0x59, 0x50, + 0x45, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x5f, 0x54, 0x4f, 0x5f, 0x43, 0x4c, + 0x4f, 0x53, 0x45, 0x10, 0x03, 0x12, 0x1a, 0x0a, 0x16, 0x54, 0x49, 0x4d, 0x45, 0x4f, 0x55, 0x54, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x48, 0x45, 0x41, 0x52, 0x54, 0x42, 0x45, 0x41, 0x54, 0x10, + 0x04, 0x42, 0x85, 0x01, 0x0a, 0x18, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x42, 0x0d, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, + 0x21, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x3b, 0x65, 0x6e, 0x75, + 0x6d, 0x73, 0xaa, 0x02, 0x17, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, + 0x41, 0x70, 0x69, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1a, 0x54, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, + 0x45, 0x6e, 0x75, 0x6d, 0x73, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x33, +} + +var ( + file_temporal_api_enums_v1_workflow_proto_rawDescOnce sync.Once + file_temporal_api_enums_v1_workflow_proto_rawDescData = file_temporal_api_enums_v1_workflow_proto_rawDesc +) + +func file_temporal_api_enums_v1_workflow_proto_rawDescGZIP() []byte { + file_temporal_api_enums_v1_workflow_proto_rawDescOnce.Do(func() { + file_temporal_api_enums_v1_workflow_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_enums_v1_workflow_proto_rawDescData) + }) + return file_temporal_api_enums_v1_workflow_proto_rawDescData +} + +var file_temporal_api_enums_v1_workflow_proto_enumTypes = make([]protoimpl.EnumInfo, 10) +var file_temporal_api_enums_v1_workflow_proto_goTypes = []any{ + (WorkflowIdReusePolicy)(0), // 0: temporal.api.enums.v1.WorkflowIdReusePolicy + (WorkflowIdConflictPolicy)(0), // 1: temporal.api.enums.v1.WorkflowIdConflictPolicy + (ParentClosePolicy)(0), // 2: temporal.api.enums.v1.ParentClosePolicy + (ContinueAsNewInitiator)(0), // 3: temporal.api.enums.v1.ContinueAsNewInitiator + (WorkflowExecutionStatus)(0), // 4: temporal.api.enums.v1.WorkflowExecutionStatus + (PendingActivityState)(0), // 5: temporal.api.enums.v1.PendingActivityState + (PendingWorkflowTaskState)(0), // 6: temporal.api.enums.v1.PendingWorkflowTaskState + (HistoryEventFilterType)(0), // 7: temporal.api.enums.v1.HistoryEventFilterType + (RetryState)(0), // 8: temporal.api.enums.v1.RetryState + (TimeoutType)(0), // 9: temporal.api.enums.v1.TimeoutType +} +var file_temporal_api_enums_v1_workflow_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_enums_v1_workflow_proto_init() } +func file_temporal_api_enums_v1_workflow_proto_init() { + if File_temporal_api_enums_v1_workflow_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_enums_v1_workflow_proto_rawDesc, + NumEnums: 10, + NumMessages: 0, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_enums_v1_workflow_proto_goTypes, + DependencyIndexes: file_temporal_api_enums_v1_workflow_proto_depIdxs, + EnumInfos: file_temporal_api_enums_v1_workflow_proto_enumTypes, + }.Build() + File_temporal_api_enums_v1_workflow_proto = out.File + file_temporal_api_enums_v1_workflow_proto_rawDesc = nil + file_temporal_api_enums_v1_workflow_proto_goTypes = nil + file_temporal_api_enums_v1_workflow_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/errordetails/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/errordetails/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..588520e63ce --- /dev/null +++ b/vendor/go.temporal.io/api/errordetails/v1/message.go-helpers.pb.go @@ -0,0 +1,620 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package errordetails + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type NotFoundFailure to the protobuf v3 wire format +func (val *NotFoundFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NotFoundFailure from the protobuf v3 wire format +func (val *NotFoundFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NotFoundFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NotFoundFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NotFoundFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NotFoundFailure + switch t := that.(type) { + case *NotFoundFailure: + that1 = t + case NotFoundFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionAlreadyStartedFailure to the protobuf v3 wire format +func (val *WorkflowExecutionAlreadyStartedFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionAlreadyStartedFailure from the protobuf v3 wire format +func (val *WorkflowExecutionAlreadyStartedFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionAlreadyStartedFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionAlreadyStartedFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionAlreadyStartedFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionAlreadyStartedFailure + switch t := that.(type) { + case *WorkflowExecutionAlreadyStartedFailure: + that1 = t + case WorkflowExecutionAlreadyStartedFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NamespaceNotActiveFailure to the protobuf v3 wire format +func (val *NamespaceNotActiveFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NamespaceNotActiveFailure from the protobuf v3 wire format +func (val *NamespaceNotActiveFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NamespaceNotActiveFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NamespaceNotActiveFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NamespaceNotActiveFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NamespaceNotActiveFailure + switch t := that.(type) { + case *NamespaceNotActiveFailure: + that1 = t + case NamespaceNotActiveFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NamespaceInvalidStateFailure to the protobuf v3 wire format +func (val *NamespaceInvalidStateFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NamespaceInvalidStateFailure from the protobuf v3 wire format +func (val *NamespaceInvalidStateFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NamespaceInvalidStateFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NamespaceInvalidStateFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NamespaceInvalidStateFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NamespaceInvalidStateFailure + switch t := that.(type) { + case *NamespaceInvalidStateFailure: + that1 = t + case NamespaceInvalidStateFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NamespaceNotFoundFailure to the protobuf v3 wire format +func (val *NamespaceNotFoundFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NamespaceNotFoundFailure from the protobuf v3 wire format +func (val *NamespaceNotFoundFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NamespaceNotFoundFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NamespaceNotFoundFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NamespaceNotFoundFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NamespaceNotFoundFailure + switch t := that.(type) { + case *NamespaceNotFoundFailure: + that1 = t + case NamespaceNotFoundFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NamespaceAlreadyExistsFailure to the protobuf v3 wire format +func (val *NamespaceAlreadyExistsFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NamespaceAlreadyExistsFailure from the protobuf v3 wire format +func (val *NamespaceAlreadyExistsFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NamespaceAlreadyExistsFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NamespaceAlreadyExistsFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NamespaceAlreadyExistsFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NamespaceAlreadyExistsFailure + switch t := that.(type) { + case *NamespaceAlreadyExistsFailure: + that1 = t + case NamespaceAlreadyExistsFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ClientVersionNotSupportedFailure to the protobuf v3 wire format +func (val *ClientVersionNotSupportedFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ClientVersionNotSupportedFailure from the protobuf v3 wire format +func (val *ClientVersionNotSupportedFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ClientVersionNotSupportedFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ClientVersionNotSupportedFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ClientVersionNotSupportedFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ClientVersionNotSupportedFailure + switch t := that.(type) { + case *ClientVersionNotSupportedFailure: + that1 = t + case ClientVersionNotSupportedFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ServerVersionNotSupportedFailure to the protobuf v3 wire format +func (val *ServerVersionNotSupportedFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ServerVersionNotSupportedFailure from the protobuf v3 wire format +func (val *ServerVersionNotSupportedFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ServerVersionNotSupportedFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ServerVersionNotSupportedFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ServerVersionNotSupportedFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ServerVersionNotSupportedFailure + switch t := that.(type) { + case *ServerVersionNotSupportedFailure: + that1 = t + case ServerVersionNotSupportedFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CancellationAlreadyRequestedFailure to the protobuf v3 wire format +func (val *CancellationAlreadyRequestedFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CancellationAlreadyRequestedFailure from the protobuf v3 wire format +func (val *CancellationAlreadyRequestedFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CancellationAlreadyRequestedFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CancellationAlreadyRequestedFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CancellationAlreadyRequestedFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CancellationAlreadyRequestedFailure + switch t := that.(type) { + case *CancellationAlreadyRequestedFailure: + that1 = t + case CancellationAlreadyRequestedFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type QueryFailedFailure to the protobuf v3 wire format +func (val *QueryFailedFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type QueryFailedFailure from the protobuf v3 wire format +func (val *QueryFailedFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *QueryFailedFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two QueryFailedFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *QueryFailedFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *QueryFailedFailure + switch t := that.(type) { + case *QueryFailedFailure: + that1 = t + case QueryFailedFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PermissionDeniedFailure to the protobuf v3 wire format +func (val *PermissionDeniedFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PermissionDeniedFailure from the protobuf v3 wire format +func (val *PermissionDeniedFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PermissionDeniedFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PermissionDeniedFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PermissionDeniedFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PermissionDeniedFailure + switch t := that.(type) { + case *PermissionDeniedFailure: + that1 = t + case PermissionDeniedFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ResourceExhaustedFailure to the protobuf v3 wire format +func (val *ResourceExhaustedFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ResourceExhaustedFailure from the protobuf v3 wire format +func (val *ResourceExhaustedFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ResourceExhaustedFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ResourceExhaustedFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ResourceExhaustedFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ResourceExhaustedFailure + switch t := that.(type) { + case *ResourceExhaustedFailure: + that1 = t + case ResourceExhaustedFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SystemWorkflowFailure to the protobuf v3 wire format +func (val *SystemWorkflowFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SystemWorkflowFailure from the protobuf v3 wire format +func (val *SystemWorkflowFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SystemWorkflowFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SystemWorkflowFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SystemWorkflowFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SystemWorkflowFailure + switch t := that.(type) { + case *SystemWorkflowFailure: + that1 = t + case SystemWorkflowFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowNotReadyFailure to the protobuf v3 wire format +func (val *WorkflowNotReadyFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowNotReadyFailure from the protobuf v3 wire format +func (val *WorkflowNotReadyFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowNotReadyFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowNotReadyFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowNotReadyFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowNotReadyFailure + switch t := that.(type) { + case *WorkflowNotReadyFailure: + that1 = t + case WorkflowNotReadyFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NewerBuildExistsFailure to the protobuf v3 wire format +func (val *NewerBuildExistsFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NewerBuildExistsFailure from the protobuf v3 wire format +func (val *NewerBuildExistsFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NewerBuildExistsFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NewerBuildExistsFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NewerBuildExistsFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NewerBuildExistsFailure + switch t := that.(type) { + case *NewerBuildExistsFailure: + that1 = t + case NewerBuildExistsFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type MultiOperationExecutionFailure to the protobuf v3 wire format +func (val *MultiOperationExecutionFailure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type MultiOperationExecutionFailure from the protobuf v3 wire format +func (val *MultiOperationExecutionFailure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *MultiOperationExecutionFailure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two MultiOperationExecutionFailure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *MultiOperationExecutionFailure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *MultiOperationExecutionFailure + switch t := that.(type) { + case *MultiOperationExecutionFailure: + that1 = t + case MultiOperationExecutionFailure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/errordetails/v1/message.pb.go b/vendor/go.temporal.io/api/errordetails/v1/message.pb.go new file mode 100644 index 00000000000..44ae0390acd --- /dev/null +++ b/vendor/go.temporal.io/api/errordetails/v1/message.pb.go @@ -0,0 +1,1354 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/errordetails/v1/message.proto + +// These error details are supplied in google.rpc.Status#details as described in "Google APIs, Error Model" (https://cloud.google.com/apis/design/errors#error_model) +// and extend standard Error Details defined in https://github.com/googleapis/googleapis/blob/master/google/rpc/error_details.proto + +package errordetails + +import ( + reflect "reflect" + sync "sync" + + v11 "go.temporal.io/api/common/v1" + v1 "go.temporal.io/api/enums/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + anypb "google.golang.org/protobuf/types/known/anypb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type NotFoundFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CurrentCluster string `protobuf:"bytes,1,opt,name=current_cluster,json=currentCluster,proto3" json:"current_cluster,omitempty"` + ActiveCluster string `protobuf:"bytes,2,opt,name=active_cluster,json=activeCluster,proto3" json:"active_cluster,omitempty"` +} + +func (x *NotFoundFailure) Reset() { + *x = NotFoundFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NotFoundFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NotFoundFailure) ProtoMessage() {} + +func (x *NotFoundFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NotFoundFailure.ProtoReflect.Descriptor instead. +func (*NotFoundFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *NotFoundFailure) GetCurrentCluster() string { + if x != nil { + return x.CurrentCluster + } + return "" +} + +func (x *NotFoundFailure) GetActiveCluster() string { + if x != nil { + return x.ActiveCluster + } + return "" +} + +type WorkflowExecutionAlreadyStartedFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + StartRequestId string `protobuf:"bytes,1,opt,name=start_request_id,json=startRequestId,proto3" json:"start_request_id,omitempty"` + RunId string `protobuf:"bytes,2,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` +} + +func (x *WorkflowExecutionAlreadyStartedFailure) Reset() { + *x = WorkflowExecutionAlreadyStartedFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionAlreadyStartedFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionAlreadyStartedFailure) ProtoMessage() {} + +func (x *WorkflowExecutionAlreadyStartedFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionAlreadyStartedFailure.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionAlreadyStartedFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *WorkflowExecutionAlreadyStartedFailure) GetStartRequestId() string { + if x != nil { + return x.StartRequestId + } + return "" +} + +func (x *WorkflowExecutionAlreadyStartedFailure) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +type NamespaceNotActiveFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + CurrentCluster string `protobuf:"bytes,2,opt,name=current_cluster,json=currentCluster,proto3" json:"current_cluster,omitempty"` + ActiveCluster string `protobuf:"bytes,3,opt,name=active_cluster,json=activeCluster,proto3" json:"active_cluster,omitempty"` +} + +func (x *NamespaceNotActiveFailure) Reset() { + *x = NamespaceNotActiveFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NamespaceNotActiveFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NamespaceNotActiveFailure) ProtoMessage() {} + +func (x *NamespaceNotActiveFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NamespaceNotActiveFailure.ProtoReflect.Descriptor instead. +func (*NamespaceNotActiveFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *NamespaceNotActiveFailure) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *NamespaceNotActiveFailure) GetCurrentCluster() string { + if x != nil { + return x.CurrentCluster + } + return "" +} + +func (x *NamespaceNotActiveFailure) GetActiveCluster() string { + if x != nil { + return x.ActiveCluster + } + return "" +} + +type NamespaceInvalidStateFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Current state of the requested namespace. + State v1.NamespaceState `protobuf:"varint,2,opt,name=state,proto3,enum=temporal.api.enums.v1.NamespaceState" json:"state,omitempty"` + // Allowed namespace states for requested operation. + // For example NAMESPACE_STATE_DELETED is forbidden for most operations but allowed for DescribeNamespace. + AllowedStates []v1.NamespaceState `protobuf:"varint,3,rep,packed,name=allowed_states,json=allowedStates,proto3,enum=temporal.api.enums.v1.NamespaceState" json:"allowed_states,omitempty"` +} + +func (x *NamespaceInvalidStateFailure) Reset() { + *x = NamespaceInvalidStateFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NamespaceInvalidStateFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NamespaceInvalidStateFailure) ProtoMessage() {} + +func (x *NamespaceInvalidStateFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NamespaceInvalidStateFailure.ProtoReflect.Descriptor instead. +func (*NamespaceInvalidStateFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *NamespaceInvalidStateFailure) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *NamespaceInvalidStateFailure) GetState() v1.NamespaceState { + if x != nil { + return x.State + } + return v1.NamespaceState(0) +} + +func (x *NamespaceInvalidStateFailure) GetAllowedStates() []v1.NamespaceState { + if x != nil { + return x.AllowedStates + } + return nil +} + +type NamespaceNotFoundFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` +} + +func (x *NamespaceNotFoundFailure) Reset() { + *x = NamespaceNotFoundFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NamespaceNotFoundFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NamespaceNotFoundFailure) ProtoMessage() {} + +func (x *NamespaceNotFoundFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NamespaceNotFoundFailure.ProtoReflect.Descriptor instead. +func (*NamespaceNotFoundFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *NamespaceNotFoundFailure) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +type NamespaceAlreadyExistsFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *NamespaceAlreadyExistsFailure) Reset() { + *x = NamespaceAlreadyExistsFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NamespaceAlreadyExistsFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NamespaceAlreadyExistsFailure) ProtoMessage() {} + +func (x *NamespaceAlreadyExistsFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NamespaceAlreadyExistsFailure.ProtoReflect.Descriptor instead. +func (*NamespaceAlreadyExistsFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{5} +} + +type ClientVersionNotSupportedFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClientVersion string `protobuf:"bytes,1,opt,name=client_version,json=clientVersion,proto3" json:"client_version,omitempty"` + ClientName string `protobuf:"bytes,2,opt,name=client_name,json=clientName,proto3" json:"client_name,omitempty"` + SupportedVersions string `protobuf:"bytes,3,opt,name=supported_versions,json=supportedVersions,proto3" json:"supported_versions,omitempty"` +} + +func (x *ClientVersionNotSupportedFailure) Reset() { + *x = ClientVersionNotSupportedFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ClientVersionNotSupportedFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ClientVersionNotSupportedFailure) ProtoMessage() {} + +func (x *ClientVersionNotSupportedFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ClientVersionNotSupportedFailure.ProtoReflect.Descriptor instead. +func (*ClientVersionNotSupportedFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{6} +} + +func (x *ClientVersionNotSupportedFailure) GetClientVersion() string { + if x != nil { + return x.ClientVersion + } + return "" +} + +func (x *ClientVersionNotSupportedFailure) GetClientName() string { + if x != nil { + return x.ClientName + } + return "" +} + +func (x *ClientVersionNotSupportedFailure) GetSupportedVersions() string { + if x != nil { + return x.SupportedVersions + } + return "" +} + +type ServerVersionNotSupportedFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ServerVersion string `protobuf:"bytes,1,opt,name=server_version,json=serverVersion,proto3" json:"server_version,omitempty"` + ClientSupportedServerVersions string `protobuf:"bytes,2,opt,name=client_supported_server_versions,json=clientSupportedServerVersions,proto3" json:"client_supported_server_versions,omitempty"` +} + +func (x *ServerVersionNotSupportedFailure) Reset() { + *x = ServerVersionNotSupportedFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ServerVersionNotSupportedFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ServerVersionNotSupportedFailure) ProtoMessage() {} + +func (x *ServerVersionNotSupportedFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ServerVersionNotSupportedFailure.ProtoReflect.Descriptor instead. +func (*ServerVersionNotSupportedFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{7} +} + +func (x *ServerVersionNotSupportedFailure) GetServerVersion() string { + if x != nil { + return x.ServerVersion + } + return "" +} + +func (x *ServerVersionNotSupportedFailure) GetClientSupportedServerVersions() string { + if x != nil { + return x.ClientSupportedServerVersions + } + return "" +} + +type CancellationAlreadyRequestedFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CancellationAlreadyRequestedFailure) Reset() { + *x = CancellationAlreadyRequestedFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CancellationAlreadyRequestedFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CancellationAlreadyRequestedFailure) ProtoMessage() {} + +func (x *CancellationAlreadyRequestedFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CancellationAlreadyRequestedFailure.ProtoReflect.Descriptor instead. +func (*CancellationAlreadyRequestedFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{8} +} + +type QueryFailedFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *QueryFailedFailure) Reset() { + *x = QueryFailedFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryFailedFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryFailedFailure) ProtoMessage() {} + +func (x *QueryFailedFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryFailedFailure.ProtoReflect.Descriptor instead. +func (*QueryFailedFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{9} +} + +type PermissionDeniedFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Reason string `protobuf:"bytes,1,opt,name=reason,proto3" json:"reason,omitempty"` +} + +func (x *PermissionDeniedFailure) Reset() { + *x = PermissionDeniedFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PermissionDeniedFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PermissionDeniedFailure) ProtoMessage() {} + +func (x *PermissionDeniedFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PermissionDeniedFailure.ProtoReflect.Descriptor instead. +func (*PermissionDeniedFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{10} +} + +func (x *PermissionDeniedFailure) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +type ResourceExhaustedFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Cause v1.ResourceExhaustedCause `protobuf:"varint,1,opt,name=cause,proto3,enum=temporal.api.enums.v1.ResourceExhaustedCause" json:"cause,omitempty"` + Scope v1.ResourceExhaustedScope `protobuf:"varint,2,opt,name=scope,proto3,enum=temporal.api.enums.v1.ResourceExhaustedScope" json:"scope,omitempty"` +} + +func (x *ResourceExhaustedFailure) Reset() { + *x = ResourceExhaustedFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResourceExhaustedFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResourceExhaustedFailure) ProtoMessage() {} + +func (x *ResourceExhaustedFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResourceExhaustedFailure.ProtoReflect.Descriptor instead. +func (*ResourceExhaustedFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{11} +} + +func (x *ResourceExhaustedFailure) GetCause() v1.ResourceExhaustedCause { + if x != nil { + return x.Cause + } + return v1.ResourceExhaustedCause(0) +} + +func (x *ResourceExhaustedFailure) GetScope() v1.ResourceExhaustedScope { + if x != nil { + return x.Scope + } + return v1.ResourceExhaustedScope(0) +} + +type SystemWorkflowFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // WorkflowId and RunId of the Temporal system workflow performing the underlying operation. + // Looking up the info of the system workflow run may help identify the issue causing the failure. + WorkflowExecution *v11.WorkflowExecution `protobuf:"bytes,1,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + // Serialized error returned by the system workflow performing the underlying operation. + WorkflowError string `protobuf:"bytes,2,opt,name=workflow_error,json=workflowError,proto3" json:"workflow_error,omitempty"` +} + +func (x *SystemWorkflowFailure) Reset() { + *x = SystemWorkflowFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SystemWorkflowFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SystemWorkflowFailure) ProtoMessage() {} + +func (x *SystemWorkflowFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SystemWorkflowFailure.ProtoReflect.Descriptor instead. +func (*SystemWorkflowFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{12} +} + +func (x *SystemWorkflowFailure) GetWorkflowExecution() *v11.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *SystemWorkflowFailure) GetWorkflowError() string { + if x != nil { + return x.WorkflowError + } + return "" +} + +type WorkflowNotReadyFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *WorkflowNotReadyFailure) Reset() { + *x = WorkflowNotReadyFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowNotReadyFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowNotReadyFailure) ProtoMessage() {} + +func (x *WorkflowNotReadyFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowNotReadyFailure.ProtoReflect.Descriptor instead. +func (*WorkflowNotReadyFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{13} +} + +type NewerBuildExistsFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The current default compatible build ID which will receive tasks + DefaultBuildId string `protobuf:"bytes,1,opt,name=default_build_id,json=defaultBuildId,proto3" json:"default_build_id,omitempty"` +} + +func (x *NewerBuildExistsFailure) Reset() { + *x = NewerBuildExistsFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NewerBuildExistsFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NewerBuildExistsFailure) ProtoMessage() {} + +func (x *NewerBuildExistsFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NewerBuildExistsFailure.ProtoReflect.Descriptor instead. +func (*NewerBuildExistsFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{14} +} + +func (x *NewerBuildExistsFailure) GetDefaultBuildId() string { + if x != nil { + return x.DefaultBuildId + } + return "" +} + +type MultiOperationExecutionFailure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // One status for each requested operation from the failed MultiOperation. The failed + // operation(s) have the same error details as if it was executed separately. All other operations have the + // status code `Aborted` and `MultiOperationExecutionAborted` is added to the details field. + Statuses []*MultiOperationExecutionFailure_OperationStatus `protobuf:"bytes,1,rep,name=statuses,proto3" json:"statuses,omitempty"` +} + +func (x *MultiOperationExecutionFailure) Reset() { + *x = MultiOperationExecutionFailure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MultiOperationExecutionFailure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MultiOperationExecutionFailure) ProtoMessage() {} + +func (x *MultiOperationExecutionFailure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[15] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MultiOperationExecutionFailure.ProtoReflect.Descriptor instead. +func (*MultiOperationExecutionFailure) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{15} +} + +func (x *MultiOperationExecutionFailure) GetStatuses() []*MultiOperationExecutionFailure_OperationStatus { + if x != nil { + return x.Statuses + } + return nil +} + +// NOTE: `OperationStatus` is modelled after +// [`google.rpc.Status`](https://github.com/googleapis/googleapis/blob/master/google/rpc/status.proto). +// +// (-- api-linter: core::0146::any=disabled +// +// aip.dev/not-precedent: details are meant to hold arbitrary payloads. --) +type MultiOperationExecutionFailure_OperationStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Code int32 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"` + Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` + Details []*anypb.Any `protobuf:"bytes,3,rep,name=details,proto3" json:"details,omitempty"` +} + +func (x *MultiOperationExecutionFailure_OperationStatus) Reset() { + *x = MultiOperationExecutionFailure_OperationStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MultiOperationExecutionFailure_OperationStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MultiOperationExecutionFailure_OperationStatus) ProtoMessage() {} + +func (x *MultiOperationExecutionFailure_OperationStatus) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_errordetails_v1_message_proto_msgTypes[16] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MultiOperationExecutionFailure_OperationStatus.ProtoReflect.Descriptor instead. +func (*MultiOperationExecutionFailure_OperationStatus) Descriptor() ([]byte, []int) { + return file_temporal_api_errordetails_v1_message_proto_rawDescGZIP(), []int{15, 0} +} + +func (x *MultiOperationExecutionFailure_OperationStatus) GetCode() int32 { + if x != nil { + return x.Code + } + return 0 +} + +func (x *MultiOperationExecutionFailure_OperationStatus) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + +func (x *MultiOperationExecutionFailure_OperationStatus) GetDetails() []*anypb.Any { + if x != nil { + return x.Details + } + return nil +} + +var File_temporal_api_errordetails_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_errordetails_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x2a, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x1c, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x64, 0x65, 0x74, 0x61, + 0x69, 0x6c, 0x73, 0x2e, 0x76, 0x31, 0x1a, 0x19, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x61, 0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x1a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x28, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x66, 0x61, 0x69, 0x6c, 0x65, + 0x64, 0x5f, 0x63, 0x61, 0x75, 0x73, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, + 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x22, 0x69, 0x0a, 0x0f, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x46, 0x61, 0x69, + 0x6c, 0x75, 0x72, 0x65, 0x12, 0x2b, 0x0a, 0x0f, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x63, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x75, 0x72, + 0x72, 0x65, 0x6e, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, + 0x0a, 0x0e, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x22, 0x71, 0x0a, 0x26, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6c, 0x72, 0x65, 0x61, + 0x64, 0x79, 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x12, + 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x19, 0x0a, 0x06, 0x72, + 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x72, 0x75, 0x6e, 0x49, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x95, 0x01, 0x0a, 0x19, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x4e, 0x6f, 0x74, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x46, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x2b, 0x0a, 0x0f, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x75, 0x72, 0x72, 0x65, + 0x6e, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, 0x0a, + 0x0e, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x6c, 0x75, 0x73, 0x74, + 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd3, 0x01, 0x0a, 0x1c, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x53, 0x74, 0x61, 0x74, 0x65, 0x46, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x50, 0x0a, 0x0e, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, + 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x52, 0x0d, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x3c, 0x0a, 0x18, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, 0x6f, + 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x12, 0x20, 0x0a, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x1f, 0x0a, + 0x1d, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, + 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x22, 0xa5, 0x01, 0x0a, + 0x20, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x74, + 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x12, 0x29, 0x0a, 0x0e, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x63, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, + 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x31, 0x0a, 0x12, + 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, + 0x64, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x9a, 0x01, 0x0a, + 0x20, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, + 0x74, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x12, 0x29, 0x0a, 0x0e, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x20, 0x63, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x5f, 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, + 0x76, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x1d, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, + 0x64, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x25, 0x0a, 0x23, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x41, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, + 0x64, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x22, 0x14, 0x0a, 0x12, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x22, 0x35, 0x0a, 0x17, + 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x6e, 0x69, 0x65, 0x64, 0x46, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x12, 0x1a, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x22, 0xac, 0x01, 0x0a, 0x18, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x78, 0x68, + 0x61, 0x75, 0x73, 0x74, 0x65, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x12, 0x47, 0x0a, 0x05, + 0x63, 0x61, 0x75, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2d, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x78, 0x68, 0x61, 0x75, 0x73, + 0x74, 0x65, 0x64, 0x43, 0x61, 0x75, 0x73, 0x65, 0x52, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x47, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x2d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x45, 0x78, 0x68, 0x61, 0x75, 0x73, 0x74, 0x65, 0x64, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x52, 0x05, 0x73, + 0x63, 0x6f, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xa0, 0x01, 0x0a, 0x15, 0x53, 0x79, 0x73, 0x74, + 0x65, 0x6d, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, 0x0a, 0x0e, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x19, 0x0a, 0x17, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x6f, + 0x74, 0x52, 0x65, 0x61, 0x64, 0x79, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x22, 0x47, 0x0a, + 0x17, 0x4e, 0x65, 0x77, 0x65, 0x72, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, + 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x12, 0x2c, 0x0a, 0x10, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, + 0x74, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0e, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x8b, 0x02, 0x0a, 0x1e, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, + 0x6c, 0x75, 0x72, 0x65, 0x12, 0x6c, 0x0a, 0x08, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, + 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x08, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x7b, 0x0a, 0x0f, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, + 0x16, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x63, 0x6f, + 0x64, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x32, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x03, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, + 0x42, 0x02, 0x68, 0x00, 0x42, 0xa7, 0x01, 0x0a, 0x1f, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x64, 0x65, 0x74, 0x61, + 0x69, 0x6c, 0x73, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, + 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x2f, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x64, 0x65, + 0x74, 0x61, 0x69, 0x6c, 0x73, 0x2f, 0x76, 0x31, 0x3b, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x64, 0x65, 0x74, + 0x61, 0x69, 0x6c, 0x73, 0xaa, 0x02, 0x1e, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, + 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, + 0x73, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x21, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, + 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65, 0x74, 0x61, 0x69, + 0x6c, 0x73, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_errordetails_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_errordetails_v1_message_proto_rawDescData = file_temporal_api_errordetails_v1_message_proto_rawDesc +) + +func file_temporal_api_errordetails_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_errordetails_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_errordetails_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_errordetails_v1_message_proto_rawDescData) + }) + return file_temporal_api_errordetails_v1_message_proto_rawDescData +} + +var file_temporal_api_errordetails_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 17) +var file_temporal_api_errordetails_v1_message_proto_goTypes = []any{ + (*NotFoundFailure)(nil), // 0: temporal.api.errordetails.v1.NotFoundFailure + (*WorkflowExecutionAlreadyStartedFailure)(nil), // 1: temporal.api.errordetails.v1.WorkflowExecutionAlreadyStartedFailure + (*NamespaceNotActiveFailure)(nil), // 2: temporal.api.errordetails.v1.NamespaceNotActiveFailure + (*NamespaceInvalidStateFailure)(nil), // 3: temporal.api.errordetails.v1.NamespaceInvalidStateFailure + (*NamespaceNotFoundFailure)(nil), // 4: temporal.api.errordetails.v1.NamespaceNotFoundFailure + (*NamespaceAlreadyExistsFailure)(nil), // 5: temporal.api.errordetails.v1.NamespaceAlreadyExistsFailure + (*ClientVersionNotSupportedFailure)(nil), // 6: temporal.api.errordetails.v1.ClientVersionNotSupportedFailure + (*ServerVersionNotSupportedFailure)(nil), // 7: temporal.api.errordetails.v1.ServerVersionNotSupportedFailure + (*CancellationAlreadyRequestedFailure)(nil), // 8: temporal.api.errordetails.v1.CancellationAlreadyRequestedFailure + (*QueryFailedFailure)(nil), // 9: temporal.api.errordetails.v1.QueryFailedFailure + (*PermissionDeniedFailure)(nil), // 10: temporal.api.errordetails.v1.PermissionDeniedFailure + (*ResourceExhaustedFailure)(nil), // 11: temporal.api.errordetails.v1.ResourceExhaustedFailure + (*SystemWorkflowFailure)(nil), // 12: temporal.api.errordetails.v1.SystemWorkflowFailure + (*WorkflowNotReadyFailure)(nil), // 13: temporal.api.errordetails.v1.WorkflowNotReadyFailure + (*NewerBuildExistsFailure)(nil), // 14: temporal.api.errordetails.v1.NewerBuildExistsFailure + (*MultiOperationExecutionFailure)(nil), // 15: temporal.api.errordetails.v1.MultiOperationExecutionFailure + (*MultiOperationExecutionFailure_OperationStatus)(nil), // 16: temporal.api.errordetails.v1.MultiOperationExecutionFailure.OperationStatus + (v1.NamespaceState)(0), // 17: temporal.api.enums.v1.NamespaceState + (v1.ResourceExhaustedCause)(0), // 18: temporal.api.enums.v1.ResourceExhaustedCause + (v1.ResourceExhaustedScope)(0), // 19: temporal.api.enums.v1.ResourceExhaustedScope + (*v11.WorkflowExecution)(nil), // 20: temporal.api.common.v1.WorkflowExecution + (*anypb.Any)(nil), // 21: google.protobuf.Any +} +var file_temporal_api_errordetails_v1_message_proto_depIdxs = []int32{ + 17, // 0: temporal.api.errordetails.v1.NamespaceInvalidStateFailure.state:type_name -> temporal.api.enums.v1.NamespaceState + 17, // 1: temporal.api.errordetails.v1.NamespaceInvalidStateFailure.allowed_states:type_name -> temporal.api.enums.v1.NamespaceState + 18, // 2: temporal.api.errordetails.v1.ResourceExhaustedFailure.cause:type_name -> temporal.api.enums.v1.ResourceExhaustedCause + 19, // 3: temporal.api.errordetails.v1.ResourceExhaustedFailure.scope:type_name -> temporal.api.enums.v1.ResourceExhaustedScope + 20, // 4: temporal.api.errordetails.v1.SystemWorkflowFailure.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 16, // 5: temporal.api.errordetails.v1.MultiOperationExecutionFailure.statuses:type_name -> temporal.api.errordetails.v1.MultiOperationExecutionFailure.OperationStatus + 21, // 6: temporal.api.errordetails.v1.MultiOperationExecutionFailure.OperationStatus.details:type_name -> google.protobuf.Any + 7, // [7:7] is the sub-list for method output_type + 7, // [7:7] is the sub-list for method input_type + 7, // [7:7] is the sub-list for extension type_name + 7, // [7:7] is the sub-list for extension extendee + 0, // [0:7] is the sub-list for field type_name +} + +func init() { file_temporal_api_errordetails_v1_message_proto_init() } +func file_temporal_api_errordetails_v1_message_proto_init() { + if File_temporal_api_errordetails_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_errordetails_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*NotFoundFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionAlreadyStartedFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*NamespaceNotActiveFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*NamespaceInvalidStateFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*NamespaceNotFoundFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*NamespaceAlreadyExistsFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*ClientVersionNotSupportedFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*ServerVersionNotSupportedFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*CancellationAlreadyRequestedFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*QueryFailedFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*PermissionDeniedFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[11].Exporter = func(v any, i int) any { + switch v := v.(*ResourceExhaustedFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[12].Exporter = func(v any, i int) any { + switch v := v.(*SystemWorkflowFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[13].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowNotReadyFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[14].Exporter = func(v any, i int) any { + switch v := v.(*NewerBuildExistsFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[15].Exporter = func(v any, i int) any { + switch v := v.(*MultiOperationExecutionFailure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_errordetails_v1_message_proto_msgTypes[16].Exporter = func(v any, i int) any { + switch v := v.(*MultiOperationExecutionFailure_OperationStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_errordetails_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 17, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_errordetails_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_errordetails_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_errordetails_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_errordetails_v1_message_proto = out.File + file_temporal_api_errordetails_v1_message_proto_rawDesc = nil + file_temporal_api_errordetails_v1_message_proto_goTypes = nil + file_temporal_api_errordetails_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/export/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/export/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..9e78652746a --- /dev/null +++ b/vendor/go.temporal.io/api/export/v1/message.go-helpers.pb.go @@ -0,0 +1,102 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package export + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type WorkflowExecution to the protobuf v3 wire format +func (val *WorkflowExecution) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecution from the protobuf v3 wire format +func (val *WorkflowExecution) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecution) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecution values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecution) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecution + switch t := that.(type) { + case *WorkflowExecution: + that1 = t + case WorkflowExecution: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutions to the protobuf v3 wire format +func (val *WorkflowExecutions) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutions from the protobuf v3 wire format +func (val *WorkflowExecutions) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutions) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutions values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutions) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutions + switch t := that.(type) { + case *WorkflowExecutions: + that1 = t + case WorkflowExecutions: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/export/v1/message.pb.go b/vendor/go.temporal.io/api/export/v1/message.pb.go new file mode 100644 index 00000000000..43ac9427af9 --- /dev/null +++ b/vendor/go.temporal.io/api/export/v1/message.pb.go @@ -0,0 +1,252 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/export/v1/message.proto + +package export + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/history/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type WorkflowExecution struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + History *v1.History `protobuf:"bytes,1,opt,name=history,proto3" json:"history,omitempty"` +} + +func (x *WorkflowExecution) Reset() { + *x = WorkflowExecution{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_export_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecution) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecution) ProtoMessage() {} + +func (x *WorkflowExecution) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_export_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecution.ProtoReflect.Descriptor instead. +func (*WorkflowExecution) Descriptor() ([]byte, []int) { + return file_temporal_api_export_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *WorkflowExecution) GetHistory() *v1.History { + if x != nil { + return x.History + } + return nil +} + +// WorkflowExecutions is used by the Cloud Export feature to deserialize +// the exported file. It encapsulates a collection of workflow execution information. +type WorkflowExecutions struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Items []*WorkflowExecution `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"` +} + +func (x *WorkflowExecutions) Reset() { + *x = WorkflowExecutions{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_export_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutions) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutions) ProtoMessage() {} + +func (x *WorkflowExecutions) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_export_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutions.ProtoReflect.Descriptor instead. +func (*WorkflowExecutions) Descriptor() ([]byte, []int) { + return file_temporal_api_export_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *WorkflowExecutions) GetItems() []*WorkflowExecution { + if x != nil { + return x.Items + } + return nil +} + +var File_temporal_api_export_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_export_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x78, + 0x70, 0x6f, 0x72, 0x74, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x16, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x65, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x1a, 0x25, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x68, 0x69, 0x73, 0x74, 0x6f, + 0x72, 0x79, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x22, 0x53, 0x0a, 0x11, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3e, 0x0a, 0x07, 0x68, 0x69, 0x73, 0x74, 0x6f, + 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, + 0x31, 0x2e, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x07, 0x68, 0x69, 0x73, 0x74, 0x6f, + 0x72, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x59, 0x0a, 0x12, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x43, 0x0a, 0x05, + 0x69, 0x74, 0x65, 0x6d, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x78, 0x70, 0x6f, 0x72, 0x74, + 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x74, 0x65, 0x6d, 0x73, 0x42, 0x02, 0x68, 0x00, 0x42, + 0x89, 0x01, 0x0a, 0x19, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x65, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x23, 0x67, 0x6f, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, + 0x2f, 0x65, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x2f, 0x76, 0x31, 0x3b, 0x65, 0x78, 0x70, 0x6f, 0x72, + 0x74, 0xaa, 0x02, 0x18, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, + 0x69, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1b, 0x54, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x45, + 0x78, 0x70, 0x6f, 0x72, 0x74, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x33, +} + +var ( + file_temporal_api_export_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_export_v1_message_proto_rawDescData = file_temporal_api_export_v1_message_proto_rawDesc +) + +func file_temporal_api_export_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_export_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_export_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_export_v1_message_proto_rawDescData) + }) + return file_temporal_api_export_v1_message_proto_rawDescData +} + +var file_temporal_api_export_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 2) +var file_temporal_api_export_v1_message_proto_goTypes = []any{ + (*WorkflowExecution)(nil), // 0: temporal.api.export.v1.WorkflowExecution + (*WorkflowExecutions)(nil), // 1: temporal.api.export.v1.WorkflowExecutions + (*v1.History)(nil), // 2: temporal.api.history.v1.History +} +var file_temporal_api_export_v1_message_proto_depIdxs = []int32{ + 2, // 0: temporal.api.export.v1.WorkflowExecution.history:type_name -> temporal.api.history.v1.History + 0, // 1: temporal.api.export.v1.WorkflowExecutions.items:type_name -> temporal.api.export.v1.WorkflowExecution + 2, // [2:2] is the sub-list for method output_type + 2, // [2:2] is the sub-list for method input_type + 2, // [2:2] is the sub-list for extension type_name + 2, // [2:2] is the sub-list for extension extendee + 0, // [0:2] is the sub-list for field type_name +} + +func init() { file_temporal_api_export_v1_message_proto_init() } +func file_temporal_api_export_v1_message_proto_init() { + if File_temporal_api_export_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_export_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecution); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_export_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_export_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 2, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_export_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_export_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_export_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_export_v1_message_proto = out.File + file_temporal_api_export_v1_message_proto_rawDesc = nil + file_temporal_api_export_v1_message_proto_goTypes = nil + file_temporal_api_export_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/failure/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/failure/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..65fb79d8e02 --- /dev/null +++ b/vendor/go.temporal.io/api/failure/v1/message.go-helpers.pb.go @@ -0,0 +1,435 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package failure + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type ApplicationFailureInfo to the protobuf v3 wire format +func (val *ApplicationFailureInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ApplicationFailureInfo from the protobuf v3 wire format +func (val *ApplicationFailureInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ApplicationFailureInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ApplicationFailureInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ApplicationFailureInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ApplicationFailureInfo + switch t := that.(type) { + case *ApplicationFailureInfo: + that1 = t + case ApplicationFailureInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TimeoutFailureInfo to the protobuf v3 wire format +func (val *TimeoutFailureInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TimeoutFailureInfo from the protobuf v3 wire format +func (val *TimeoutFailureInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TimeoutFailureInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TimeoutFailureInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TimeoutFailureInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TimeoutFailureInfo + switch t := that.(type) { + case *TimeoutFailureInfo: + that1 = t + case TimeoutFailureInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CanceledFailureInfo to the protobuf v3 wire format +func (val *CanceledFailureInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CanceledFailureInfo from the protobuf v3 wire format +func (val *CanceledFailureInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CanceledFailureInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CanceledFailureInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CanceledFailureInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CanceledFailureInfo + switch t := that.(type) { + case *CanceledFailureInfo: + that1 = t + case CanceledFailureInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TerminatedFailureInfo to the protobuf v3 wire format +func (val *TerminatedFailureInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TerminatedFailureInfo from the protobuf v3 wire format +func (val *TerminatedFailureInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TerminatedFailureInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TerminatedFailureInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TerminatedFailureInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TerminatedFailureInfo + switch t := that.(type) { + case *TerminatedFailureInfo: + that1 = t + case TerminatedFailureInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ServerFailureInfo to the protobuf v3 wire format +func (val *ServerFailureInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ServerFailureInfo from the protobuf v3 wire format +func (val *ServerFailureInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ServerFailureInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ServerFailureInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ServerFailureInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ServerFailureInfo + switch t := that.(type) { + case *ServerFailureInfo: + that1 = t + case ServerFailureInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ResetWorkflowFailureInfo to the protobuf v3 wire format +func (val *ResetWorkflowFailureInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ResetWorkflowFailureInfo from the protobuf v3 wire format +func (val *ResetWorkflowFailureInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ResetWorkflowFailureInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ResetWorkflowFailureInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ResetWorkflowFailureInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ResetWorkflowFailureInfo + switch t := that.(type) { + case *ResetWorkflowFailureInfo: + that1 = t + case ResetWorkflowFailureInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ActivityFailureInfo to the protobuf v3 wire format +func (val *ActivityFailureInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ActivityFailureInfo from the protobuf v3 wire format +func (val *ActivityFailureInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ActivityFailureInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ActivityFailureInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ActivityFailureInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ActivityFailureInfo + switch t := that.(type) { + case *ActivityFailureInfo: + that1 = t + case ActivityFailureInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ChildWorkflowExecutionFailureInfo to the protobuf v3 wire format +func (val *ChildWorkflowExecutionFailureInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ChildWorkflowExecutionFailureInfo from the protobuf v3 wire format +func (val *ChildWorkflowExecutionFailureInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ChildWorkflowExecutionFailureInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ChildWorkflowExecutionFailureInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ChildWorkflowExecutionFailureInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ChildWorkflowExecutionFailureInfo + switch t := that.(type) { + case *ChildWorkflowExecutionFailureInfo: + that1 = t + case ChildWorkflowExecutionFailureInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NexusOperationFailureInfo to the protobuf v3 wire format +func (val *NexusOperationFailureInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NexusOperationFailureInfo from the protobuf v3 wire format +func (val *NexusOperationFailureInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NexusOperationFailureInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NexusOperationFailureInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NexusOperationFailureInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NexusOperationFailureInfo + switch t := that.(type) { + case *NexusOperationFailureInfo: + that1 = t + case NexusOperationFailureInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Failure to the protobuf v3 wire format +func (val *Failure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Failure from the protobuf v3 wire format +func (val *Failure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Failure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Failure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Failure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Failure + switch t := that.(type) { + case *Failure: + that1 = t + case Failure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type MultiOperationExecutionAborted to the protobuf v3 wire format +func (val *MultiOperationExecutionAborted) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type MultiOperationExecutionAborted from the protobuf v3 wire format +func (val *MultiOperationExecutionAborted) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *MultiOperationExecutionAborted) Size() int { + return proto.Size(val) +} + +// Equal returns whether two MultiOperationExecutionAborted values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *MultiOperationExecutionAborted) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *MultiOperationExecutionAborted + switch t := that.(type) { + case *MultiOperationExecutionAborted: + that1 = t + case MultiOperationExecutionAborted: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/failure/v1/message.pb.go b/vendor/go.temporal.io/api/failure/v1/message.pb.go new file mode 100644 index 00000000000..ae31ea496f6 --- /dev/null +++ b/vendor/go.temporal.io/api/failure/v1/message.pb.go @@ -0,0 +1,1327 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/failure/v1/message.proto + +package failure + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/common/v1" + v11 "go.temporal.io/api/enums/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + durationpb "google.golang.org/protobuf/types/known/durationpb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type ApplicationFailureInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Type string `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"` + NonRetryable bool `protobuf:"varint,2,opt,name=non_retryable,json=nonRetryable,proto3" json:"non_retryable,omitempty"` + Details *v1.Payloads `protobuf:"bytes,3,opt,name=details,proto3" json:"details,omitempty"` + // next_retry_delay can be used by the client to override the activity + // retry interval calculated by the retry policy. Retry attempts will + // still be subject to the maximum retries limit and total time limit + // defined by the policy. + NextRetryDelay *durationpb.Duration `protobuf:"bytes,4,opt,name=next_retry_delay,json=nextRetryDelay,proto3" json:"next_retry_delay,omitempty"` +} + +func (x *ApplicationFailureInfo) Reset() { + *x = ApplicationFailureInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ApplicationFailureInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ApplicationFailureInfo) ProtoMessage() {} + +func (x *ApplicationFailureInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ApplicationFailureInfo.ProtoReflect.Descriptor instead. +func (*ApplicationFailureInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_failure_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *ApplicationFailureInfo) GetType() string { + if x != nil { + return x.Type + } + return "" +} + +func (x *ApplicationFailureInfo) GetNonRetryable() bool { + if x != nil { + return x.NonRetryable + } + return false +} + +func (x *ApplicationFailureInfo) GetDetails() *v1.Payloads { + if x != nil { + return x.Details + } + return nil +} + +func (x *ApplicationFailureInfo) GetNextRetryDelay() *durationpb.Duration { + if x != nil { + return x.NextRetryDelay + } + return nil +} + +type TimeoutFailureInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TimeoutType v11.TimeoutType `protobuf:"varint,1,opt,name=timeout_type,json=timeoutType,proto3,enum=temporal.api.enums.v1.TimeoutType" json:"timeout_type,omitempty"` + LastHeartbeatDetails *v1.Payloads `protobuf:"bytes,2,opt,name=last_heartbeat_details,json=lastHeartbeatDetails,proto3" json:"last_heartbeat_details,omitempty"` +} + +func (x *TimeoutFailureInfo) Reset() { + *x = TimeoutFailureInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TimeoutFailureInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimeoutFailureInfo) ProtoMessage() {} + +func (x *TimeoutFailureInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TimeoutFailureInfo.ProtoReflect.Descriptor instead. +func (*TimeoutFailureInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_failure_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *TimeoutFailureInfo) GetTimeoutType() v11.TimeoutType { + if x != nil { + return x.TimeoutType + } + return v11.TimeoutType(0) +} + +func (x *TimeoutFailureInfo) GetLastHeartbeatDetails() *v1.Payloads { + if x != nil { + return x.LastHeartbeatDetails + } + return nil +} + +type CanceledFailureInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Details *v1.Payloads `protobuf:"bytes,1,opt,name=details,proto3" json:"details,omitempty"` +} + +func (x *CanceledFailureInfo) Reset() { + *x = CanceledFailureInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CanceledFailureInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CanceledFailureInfo) ProtoMessage() {} + +func (x *CanceledFailureInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CanceledFailureInfo.ProtoReflect.Descriptor instead. +func (*CanceledFailureInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_failure_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *CanceledFailureInfo) GetDetails() *v1.Payloads { + if x != nil { + return x.Details + } + return nil +} + +type TerminatedFailureInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *TerminatedFailureInfo) Reset() { + *x = TerminatedFailureInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TerminatedFailureInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TerminatedFailureInfo) ProtoMessage() {} + +func (x *TerminatedFailureInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TerminatedFailureInfo.ProtoReflect.Descriptor instead. +func (*TerminatedFailureInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_failure_v1_message_proto_rawDescGZIP(), []int{3} +} + +type ServerFailureInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + NonRetryable bool `protobuf:"varint,1,opt,name=non_retryable,json=nonRetryable,proto3" json:"non_retryable,omitempty"` +} + +func (x *ServerFailureInfo) Reset() { + *x = ServerFailureInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ServerFailureInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ServerFailureInfo) ProtoMessage() {} + +func (x *ServerFailureInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ServerFailureInfo.ProtoReflect.Descriptor instead. +func (*ServerFailureInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_failure_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *ServerFailureInfo) GetNonRetryable() bool { + if x != nil { + return x.NonRetryable + } + return false +} + +type ResetWorkflowFailureInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + LastHeartbeatDetails *v1.Payloads `protobuf:"bytes,1,opt,name=last_heartbeat_details,json=lastHeartbeatDetails,proto3" json:"last_heartbeat_details,omitempty"` +} + +func (x *ResetWorkflowFailureInfo) Reset() { + *x = ResetWorkflowFailureInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResetWorkflowFailureInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResetWorkflowFailureInfo) ProtoMessage() {} + +func (x *ResetWorkflowFailureInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResetWorkflowFailureInfo.ProtoReflect.Descriptor instead. +func (*ResetWorkflowFailureInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_failure_v1_message_proto_rawDescGZIP(), []int{5} +} + +func (x *ResetWorkflowFailureInfo) GetLastHeartbeatDetails() *v1.Payloads { + if x != nil { + return x.LastHeartbeatDetails + } + return nil +} + +type ActivityFailureInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + StartedEventId int64 `protobuf:"varint,2,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` + ActivityType *v1.ActivityType `protobuf:"bytes,4,opt,name=activity_type,json=activityType,proto3" json:"activity_type,omitempty"` + ActivityId string `protobuf:"bytes,5,opt,name=activity_id,json=activityId,proto3" json:"activity_id,omitempty"` + RetryState v11.RetryState `protobuf:"varint,6,opt,name=retry_state,json=retryState,proto3,enum=temporal.api.enums.v1.RetryState" json:"retry_state,omitempty"` +} + +func (x *ActivityFailureInfo) Reset() { + *x = ActivityFailureInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActivityFailureInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActivityFailureInfo) ProtoMessage() {} + +func (x *ActivityFailureInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActivityFailureInfo.ProtoReflect.Descriptor instead. +func (*ActivityFailureInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_failure_v1_message_proto_rawDescGZIP(), []int{6} +} + +func (x *ActivityFailureInfo) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *ActivityFailureInfo) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +func (x *ActivityFailureInfo) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *ActivityFailureInfo) GetActivityType() *v1.ActivityType { + if x != nil { + return x.ActivityType + } + return nil +} + +func (x *ActivityFailureInfo) GetActivityId() string { + if x != nil { + return x.ActivityId + } + return "" +} + +func (x *ActivityFailureInfo) GetRetryState() v11.RetryState { + if x != nil { + return x.RetryState + } + return v11.RetryState(0) +} + +type ChildWorkflowExecutionFailureInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + WorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,2,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + WorkflowType *v1.WorkflowType `protobuf:"bytes,3,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + InitiatedEventId int64 `protobuf:"varint,4,opt,name=initiated_event_id,json=initiatedEventId,proto3" json:"initiated_event_id,omitempty"` + StartedEventId int64 `protobuf:"varint,5,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` + RetryState v11.RetryState `protobuf:"varint,6,opt,name=retry_state,json=retryState,proto3,enum=temporal.api.enums.v1.RetryState" json:"retry_state,omitempty"` +} + +func (x *ChildWorkflowExecutionFailureInfo) Reset() { + *x = ChildWorkflowExecutionFailureInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChildWorkflowExecutionFailureInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChildWorkflowExecutionFailureInfo) ProtoMessage() {} + +func (x *ChildWorkflowExecutionFailureInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ChildWorkflowExecutionFailureInfo.ProtoReflect.Descriptor instead. +func (*ChildWorkflowExecutionFailureInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_failure_v1_message_proto_rawDescGZIP(), []int{7} +} + +func (x *ChildWorkflowExecutionFailureInfo) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ChildWorkflowExecutionFailureInfo) GetWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *ChildWorkflowExecutionFailureInfo) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *ChildWorkflowExecutionFailureInfo) GetInitiatedEventId() int64 { + if x != nil { + return x.InitiatedEventId + } + return 0 +} + +func (x *ChildWorkflowExecutionFailureInfo) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +func (x *ChildWorkflowExecutionFailureInfo) GetRetryState() v11.RetryState { + if x != nil { + return x.RetryState + } + return v11.RetryState(0) +} + +type NexusOperationFailureInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The NexusOperationScheduled event ID. + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // Endpoint name. + Endpoint string `protobuf:"bytes,2,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + // Service name. + Service string `protobuf:"bytes,3,opt,name=service,proto3" json:"service,omitempty"` + // Operation name. + Operation string `protobuf:"bytes,4,opt,name=operation,proto3" json:"operation,omitempty"` + // Operation ID - may be empty if the operation completed synchronously. + OperationId string `protobuf:"bytes,5,opt,name=operation_id,json=operationId,proto3" json:"operation_id,omitempty"` +} + +func (x *NexusOperationFailureInfo) Reset() { + *x = NexusOperationFailureInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NexusOperationFailureInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NexusOperationFailureInfo) ProtoMessage() {} + +func (x *NexusOperationFailureInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NexusOperationFailureInfo.ProtoReflect.Descriptor instead. +func (*NexusOperationFailureInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_failure_v1_message_proto_rawDescGZIP(), []int{8} +} + +func (x *NexusOperationFailureInfo) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *NexusOperationFailureInfo) GetEndpoint() string { + if x != nil { + return x.Endpoint + } + return "" +} + +func (x *NexusOperationFailureInfo) GetService() string { + if x != nil { + return x.Service + } + return "" +} + +func (x *NexusOperationFailureInfo) GetOperation() string { + if x != nil { + return x.Operation + } + return "" +} + +func (x *NexusOperationFailureInfo) GetOperationId() string { + if x != nil { + return x.OperationId + } + return "" +} + +type Failure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Message string `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"` + // The source this Failure originated in, e.g. TypeScriptSDK / JavaSDK + // In some SDKs this is used to rehydrate the stack trace into an exception object. + Source string `protobuf:"bytes,2,opt,name=source,proto3" json:"source,omitempty"` + StackTrace string `protobuf:"bytes,3,opt,name=stack_trace,json=stackTrace,proto3" json:"stack_trace,omitempty"` + // Alternative way to supply `message` and `stack_trace` and possibly other attributes, used for encryption of + // errors originating in user code which might contain sensitive information. + // The `encoded_attributes` Payload could represent any serializable object, e.g. JSON object or a `Failure` proto + // message. + // + // SDK authors: + // - The SDK should provide a default `encodeFailureAttributes` and `decodeFailureAttributes` implementation that: + // - Uses a JSON object to represent `{ message, stack_trace }`. + // - Overwrites the original message with "Encoded failure" to indicate that more information could be extracted. + // - Overwrites the original stack_trace with an empty string. + // - The resulting JSON object is converted to Payload using the default PayloadConverter and should be processed + // by the user-provided PayloadCodec + // + // - If there's demand, we could allow overriding the default SDK implementation to encode other opaque Failure attributes. + // (-- api-linter: core::0203::optional=disabled --) + EncodedAttributes *v1.Payload `protobuf:"bytes,20,opt,name=encoded_attributes,json=encodedAttributes,proto3" json:"encoded_attributes,omitempty"` + Cause *Failure `protobuf:"bytes,4,opt,name=cause,proto3" json:"cause,omitempty"` + // Types that are assignable to FailureInfo: + // + // *Failure_ApplicationFailureInfo + // *Failure_TimeoutFailureInfo + // *Failure_CanceledFailureInfo + // *Failure_TerminatedFailureInfo + // *Failure_ServerFailureInfo + // *Failure_ResetWorkflowFailureInfo + // *Failure_ActivityFailureInfo + // *Failure_ChildWorkflowExecutionFailureInfo + // *Failure_NexusOperationExecutionFailureInfo + FailureInfo isFailure_FailureInfo `protobuf_oneof:"failure_info"` +} + +func (x *Failure) Reset() { + *x = Failure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Failure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Failure) ProtoMessage() {} + +func (x *Failure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Failure.ProtoReflect.Descriptor instead. +func (*Failure) Descriptor() ([]byte, []int) { + return file_temporal_api_failure_v1_message_proto_rawDescGZIP(), []int{9} +} + +func (x *Failure) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + +func (x *Failure) GetSource() string { + if x != nil { + return x.Source + } + return "" +} + +func (x *Failure) GetStackTrace() string { + if x != nil { + return x.StackTrace + } + return "" +} + +func (x *Failure) GetEncodedAttributes() *v1.Payload { + if x != nil { + return x.EncodedAttributes + } + return nil +} + +func (x *Failure) GetCause() *Failure { + if x != nil { + return x.Cause + } + return nil +} + +func (m *Failure) GetFailureInfo() isFailure_FailureInfo { + if m != nil { + return m.FailureInfo + } + return nil +} + +func (x *Failure) GetApplicationFailureInfo() *ApplicationFailureInfo { + if x, ok := x.GetFailureInfo().(*Failure_ApplicationFailureInfo); ok { + return x.ApplicationFailureInfo + } + return nil +} + +func (x *Failure) GetTimeoutFailureInfo() *TimeoutFailureInfo { + if x, ok := x.GetFailureInfo().(*Failure_TimeoutFailureInfo); ok { + return x.TimeoutFailureInfo + } + return nil +} + +func (x *Failure) GetCanceledFailureInfo() *CanceledFailureInfo { + if x, ok := x.GetFailureInfo().(*Failure_CanceledFailureInfo); ok { + return x.CanceledFailureInfo + } + return nil +} + +func (x *Failure) GetTerminatedFailureInfo() *TerminatedFailureInfo { + if x, ok := x.GetFailureInfo().(*Failure_TerminatedFailureInfo); ok { + return x.TerminatedFailureInfo + } + return nil +} + +func (x *Failure) GetServerFailureInfo() *ServerFailureInfo { + if x, ok := x.GetFailureInfo().(*Failure_ServerFailureInfo); ok { + return x.ServerFailureInfo + } + return nil +} + +func (x *Failure) GetResetWorkflowFailureInfo() *ResetWorkflowFailureInfo { + if x, ok := x.GetFailureInfo().(*Failure_ResetWorkflowFailureInfo); ok { + return x.ResetWorkflowFailureInfo + } + return nil +} + +func (x *Failure) GetActivityFailureInfo() *ActivityFailureInfo { + if x, ok := x.GetFailureInfo().(*Failure_ActivityFailureInfo); ok { + return x.ActivityFailureInfo + } + return nil +} + +func (x *Failure) GetChildWorkflowExecutionFailureInfo() *ChildWorkflowExecutionFailureInfo { + if x, ok := x.GetFailureInfo().(*Failure_ChildWorkflowExecutionFailureInfo); ok { + return x.ChildWorkflowExecutionFailureInfo + } + return nil +} + +func (x *Failure) GetNexusOperationExecutionFailureInfo() *NexusOperationFailureInfo { + if x, ok := x.GetFailureInfo().(*Failure_NexusOperationExecutionFailureInfo); ok { + return x.NexusOperationExecutionFailureInfo + } + return nil +} + +type isFailure_FailureInfo interface { + isFailure_FailureInfo() +} + +type Failure_ApplicationFailureInfo struct { + ApplicationFailureInfo *ApplicationFailureInfo `protobuf:"bytes,5,opt,name=application_failure_info,json=applicationFailureInfo,proto3,oneof"` +} + +type Failure_TimeoutFailureInfo struct { + TimeoutFailureInfo *TimeoutFailureInfo `protobuf:"bytes,6,opt,name=timeout_failure_info,json=timeoutFailureInfo,proto3,oneof"` +} + +type Failure_CanceledFailureInfo struct { + CanceledFailureInfo *CanceledFailureInfo `protobuf:"bytes,7,opt,name=canceled_failure_info,json=canceledFailureInfo,proto3,oneof"` +} + +type Failure_TerminatedFailureInfo struct { + TerminatedFailureInfo *TerminatedFailureInfo `protobuf:"bytes,8,opt,name=terminated_failure_info,json=terminatedFailureInfo,proto3,oneof"` +} + +type Failure_ServerFailureInfo struct { + ServerFailureInfo *ServerFailureInfo `protobuf:"bytes,9,opt,name=server_failure_info,json=serverFailureInfo,proto3,oneof"` +} + +type Failure_ResetWorkflowFailureInfo struct { + ResetWorkflowFailureInfo *ResetWorkflowFailureInfo `protobuf:"bytes,10,opt,name=reset_workflow_failure_info,json=resetWorkflowFailureInfo,proto3,oneof"` +} + +type Failure_ActivityFailureInfo struct { + ActivityFailureInfo *ActivityFailureInfo `protobuf:"bytes,11,opt,name=activity_failure_info,json=activityFailureInfo,proto3,oneof"` +} + +type Failure_ChildWorkflowExecutionFailureInfo struct { + ChildWorkflowExecutionFailureInfo *ChildWorkflowExecutionFailureInfo `protobuf:"bytes,12,opt,name=child_workflow_execution_failure_info,json=childWorkflowExecutionFailureInfo,proto3,oneof"` +} + +type Failure_NexusOperationExecutionFailureInfo struct { + NexusOperationExecutionFailureInfo *NexusOperationFailureInfo `protobuf:"bytes,13,opt,name=nexus_operation_execution_failure_info,json=nexusOperationExecutionFailureInfo,proto3,oneof"` +} + +func (*Failure_ApplicationFailureInfo) isFailure_FailureInfo() {} + +func (*Failure_TimeoutFailureInfo) isFailure_FailureInfo() {} + +func (*Failure_CanceledFailureInfo) isFailure_FailureInfo() {} + +func (*Failure_TerminatedFailureInfo) isFailure_FailureInfo() {} + +func (*Failure_ServerFailureInfo) isFailure_FailureInfo() {} + +func (*Failure_ResetWorkflowFailureInfo) isFailure_FailureInfo() {} + +func (*Failure_ActivityFailureInfo) isFailure_FailureInfo() {} + +func (*Failure_ChildWorkflowExecutionFailureInfo) isFailure_FailureInfo() {} + +func (*Failure_NexusOperationExecutionFailureInfo) isFailure_FailureInfo() {} + +type MultiOperationExecutionAborted struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *MultiOperationExecutionAborted) Reset() { + *x = MultiOperationExecutionAborted{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MultiOperationExecutionAborted) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MultiOperationExecutionAborted) ProtoMessage() {} + +func (x *MultiOperationExecutionAborted) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_failure_v1_message_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MultiOperationExecutionAborted.ProtoReflect.Descriptor instead. +func (*MultiOperationExecutionAborted) Descriptor() ([]byte, []int) { + return file_temporal_api_failure_v1_message_proto_rawDescGZIP(), []int{10} +} + +var File_temporal_api_failure_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_failure_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x25, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x66, 0x61, + 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x17, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x1a, 0x24, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x22, 0xe2, 0x01, 0x0a, 0x16, 0x41, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x16, 0x0a, + 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, 0x0a, 0x0d, 0x6e, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, + 0x61, 0x62, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x6e, 0x6f, 0x6e, 0x52, 0x65, + 0x74, 0x72, 0x79, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x64, + 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x07, 0x64, 0x65, 0x74, + 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x10, 0x6e, 0x65, 0x78, 0x74, 0x5f, + 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x6e, 0x65, 0x78, 0x74, + 0x52, 0x65, 0x74, 0x72, 0x79, 0x44, 0x65, 0x6c, 0x61, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0xbb, 0x01, + 0x0a, 0x12, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, + 0x6e, 0x66, 0x6f, 0x12, 0x49, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x22, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, + 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, 0x74, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5a, 0x0a, 0x16, 0x6c, 0x61, + 0x73, 0x74, 0x5f, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x5f, 0x64, 0x65, 0x74, + 0x61, 0x69, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x48, + 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x55, 0x0a, 0x13, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x46, 0x61, + 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x3e, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, + 0x69, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x17, 0x0a, 0x15, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, + 0x74, 0x65, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x3c, 0x0a, + 0x11, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x27, 0x0a, 0x0d, 0x6e, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, 0x61, 0x62, 0x6c, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x6e, 0x6f, 0x6e, 0x52, 0x65, 0x74, 0x72, 0x79, + 0x61, 0x62, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x76, 0x0a, 0x18, 0x52, 0x65, 0x73, 0x65, + 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, + 0x6e, 0x66, 0x6f, 0x12, 0x5a, 0x0a, 0x16, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x68, 0x65, 0x61, 0x72, 0x74, + 0x62, 0x65, 0x61, 0x74, 0x5f, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, + 0x64, 0x73, 0x52, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, + 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd1, 0x02, 0x0a, 0x13, 0x41, + 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, + 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, + 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x61, + 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, + 0x79, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, + 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, + 0x74, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x61, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x79, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x46, 0x0a, 0x0b, 0x72, 0x65, 0x74, + 0x72, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, + 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x9a, + 0x03, 0x0a, 0x21, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, + 0x66, 0x6f, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x69, 0x6e, 0x69, 0x74, 0x69, + 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x10, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, + 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x46, 0x0a, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, + 0x74, 0x72, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd4, 0x01, 0x0a, 0x19, 0x4e, 0x65, 0x78, 0x75, + 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, + 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, + 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x25, 0x0a, 0x0c, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x22, 0x9a, 0x0a, 0x0a, 0x07, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x12, + 0x1c, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1a, 0x0a, 0x06, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x63, 0x6b, 0x5f, + 0x74, 0x72, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x63, + 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x52, 0x0a, 0x12, 0x65, 0x6e, + 0x63, 0x6f, 0x64, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, + 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x11, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x65, 0x64, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x63, 0x61, + 0x75, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x6f, 0x0a, 0x18, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, + 0x66, 0x6f, 0x48, 0x00, 0x52, 0x16, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x63, 0x0a, 0x14, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x48, 0x00, 0x52, 0x12, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, + 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x66, 0x0a, 0x15, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x46, 0x61, 0x69, + 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x48, 0x00, 0x52, 0x13, 0x63, 0x61, 0x6e, 0x63, + 0x65, 0x6c, 0x65, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x6c, 0x0a, 0x17, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x5f, + 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x08, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x72, 0x6d, 0x69, + 0x6e, 0x61, 0x74, 0x65, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x48, + 0x00, 0x52, 0x15, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x46, 0x61, 0x69, 0x6c, + 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x60, 0x0a, 0x13, 0x73, 0x65, + 0x72, 0x76, 0x65, 0x72, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, + 0x48, 0x00, 0x52, 0x11, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x76, 0x0a, 0x1b, 0x72, 0x65, 0x73, 0x65, 0x74, + 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x48, 0x00, 0x52, 0x18, 0x72, + 0x65, 0x73, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x46, 0x61, 0x69, 0x6c, + 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x66, 0x0a, 0x15, 0x61, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x5f, 0x69, 0x6e, + 0x66, 0x6f, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x49, 0x6e, 0x66, 0x6f, 0x48, 0x00, 0x52, 0x13, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x46, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x92, 0x01, + 0x0a, 0x25, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x49, 0x6e, 0x66, 0x6f, 0x48, 0x00, 0x52, 0x21, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, + 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x8c, 0x01, 0x0a, 0x26, + 0x6e, 0x65, 0x78, 0x75, 0x73, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x5f, + 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x48, 0x00, 0x52, 0x22, 0x6e, + 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x42, + 0x02, 0x68, 0x00, 0x42, 0x0e, 0x0a, 0x0c, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x5f, 0x69, + 0x6e, 0x66, 0x6f, 0x22, 0x20, 0x0a, 0x1e, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x62, 0x6f, 0x72, + 0x74, 0x65, 0x64, 0x42, 0x8e, 0x01, 0x0a, 0x1a, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, + 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, + 0x5a, 0x25, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2f, 0x76, 0x31, 0x3b, 0x66, 0x61, + 0x69, 0x6c, 0x75, 0x72, 0x65, 0xaa, 0x02, 0x19, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, + 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x56, 0x31, 0xea, + 0x02, 0x1c, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, + 0x69, 0x3a, 0x3a, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_failure_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_failure_v1_message_proto_rawDescData = file_temporal_api_failure_v1_message_proto_rawDesc +) + +func file_temporal_api_failure_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_failure_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_failure_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_failure_v1_message_proto_rawDescData) + }) + return file_temporal_api_failure_v1_message_proto_rawDescData +} + +var file_temporal_api_failure_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 11) +var file_temporal_api_failure_v1_message_proto_goTypes = []any{ + (*ApplicationFailureInfo)(nil), // 0: temporal.api.failure.v1.ApplicationFailureInfo + (*TimeoutFailureInfo)(nil), // 1: temporal.api.failure.v1.TimeoutFailureInfo + (*CanceledFailureInfo)(nil), // 2: temporal.api.failure.v1.CanceledFailureInfo + (*TerminatedFailureInfo)(nil), // 3: temporal.api.failure.v1.TerminatedFailureInfo + (*ServerFailureInfo)(nil), // 4: temporal.api.failure.v1.ServerFailureInfo + (*ResetWorkflowFailureInfo)(nil), // 5: temporal.api.failure.v1.ResetWorkflowFailureInfo + (*ActivityFailureInfo)(nil), // 6: temporal.api.failure.v1.ActivityFailureInfo + (*ChildWorkflowExecutionFailureInfo)(nil), // 7: temporal.api.failure.v1.ChildWorkflowExecutionFailureInfo + (*NexusOperationFailureInfo)(nil), // 8: temporal.api.failure.v1.NexusOperationFailureInfo + (*Failure)(nil), // 9: temporal.api.failure.v1.Failure + (*MultiOperationExecutionAborted)(nil), // 10: temporal.api.failure.v1.MultiOperationExecutionAborted + (*v1.Payloads)(nil), // 11: temporal.api.common.v1.Payloads + (*durationpb.Duration)(nil), // 12: google.protobuf.Duration + (v11.TimeoutType)(0), // 13: temporal.api.enums.v1.TimeoutType + (*v1.ActivityType)(nil), // 14: temporal.api.common.v1.ActivityType + (v11.RetryState)(0), // 15: temporal.api.enums.v1.RetryState + (*v1.WorkflowExecution)(nil), // 16: temporal.api.common.v1.WorkflowExecution + (*v1.WorkflowType)(nil), // 17: temporal.api.common.v1.WorkflowType + (*v1.Payload)(nil), // 18: temporal.api.common.v1.Payload +} +var file_temporal_api_failure_v1_message_proto_depIdxs = []int32{ + 11, // 0: temporal.api.failure.v1.ApplicationFailureInfo.details:type_name -> temporal.api.common.v1.Payloads + 12, // 1: temporal.api.failure.v1.ApplicationFailureInfo.next_retry_delay:type_name -> google.protobuf.Duration + 13, // 2: temporal.api.failure.v1.TimeoutFailureInfo.timeout_type:type_name -> temporal.api.enums.v1.TimeoutType + 11, // 3: temporal.api.failure.v1.TimeoutFailureInfo.last_heartbeat_details:type_name -> temporal.api.common.v1.Payloads + 11, // 4: temporal.api.failure.v1.CanceledFailureInfo.details:type_name -> temporal.api.common.v1.Payloads + 11, // 5: temporal.api.failure.v1.ResetWorkflowFailureInfo.last_heartbeat_details:type_name -> temporal.api.common.v1.Payloads + 14, // 6: temporal.api.failure.v1.ActivityFailureInfo.activity_type:type_name -> temporal.api.common.v1.ActivityType + 15, // 7: temporal.api.failure.v1.ActivityFailureInfo.retry_state:type_name -> temporal.api.enums.v1.RetryState + 16, // 8: temporal.api.failure.v1.ChildWorkflowExecutionFailureInfo.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 17, // 9: temporal.api.failure.v1.ChildWorkflowExecutionFailureInfo.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 15, // 10: temporal.api.failure.v1.ChildWorkflowExecutionFailureInfo.retry_state:type_name -> temporal.api.enums.v1.RetryState + 18, // 11: temporal.api.failure.v1.Failure.encoded_attributes:type_name -> temporal.api.common.v1.Payload + 9, // 12: temporal.api.failure.v1.Failure.cause:type_name -> temporal.api.failure.v1.Failure + 0, // 13: temporal.api.failure.v1.Failure.application_failure_info:type_name -> temporal.api.failure.v1.ApplicationFailureInfo + 1, // 14: temporal.api.failure.v1.Failure.timeout_failure_info:type_name -> temporal.api.failure.v1.TimeoutFailureInfo + 2, // 15: temporal.api.failure.v1.Failure.canceled_failure_info:type_name -> temporal.api.failure.v1.CanceledFailureInfo + 3, // 16: temporal.api.failure.v1.Failure.terminated_failure_info:type_name -> temporal.api.failure.v1.TerminatedFailureInfo + 4, // 17: temporal.api.failure.v1.Failure.server_failure_info:type_name -> temporal.api.failure.v1.ServerFailureInfo + 5, // 18: temporal.api.failure.v1.Failure.reset_workflow_failure_info:type_name -> temporal.api.failure.v1.ResetWorkflowFailureInfo + 6, // 19: temporal.api.failure.v1.Failure.activity_failure_info:type_name -> temporal.api.failure.v1.ActivityFailureInfo + 7, // 20: temporal.api.failure.v1.Failure.child_workflow_execution_failure_info:type_name -> temporal.api.failure.v1.ChildWorkflowExecutionFailureInfo + 8, // 21: temporal.api.failure.v1.Failure.nexus_operation_execution_failure_info:type_name -> temporal.api.failure.v1.NexusOperationFailureInfo + 22, // [22:22] is the sub-list for method output_type + 22, // [22:22] is the sub-list for method input_type + 22, // [22:22] is the sub-list for extension type_name + 22, // [22:22] is the sub-list for extension extendee + 0, // [0:22] is the sub-list for field type_name +} + +func init() { file_temporal_api_failure_v1_message_proto_init() } +func file_temporal_api_failure_v1_message_proto_init() { + if File_temporal_api_failure_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_failure_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*ApplicationFailureInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_failure_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*TimeoutFailureInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_failure_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*CanceledFailureInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_failure_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*TerminatedFailureInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_failure_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*ServerFailureInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_failure_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*ResetWorkflowFailureInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_failure_v1_message_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*ActivityFailureInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_failure_v1_message_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*ChildWorkflowExecutionFailureInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_failure_v1_message_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*NexusOperationFailureInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_failure_v1_message_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*Failure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_failure_v1_message_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*MultiOperationExecutionAborted); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_temporal_api_failure_v1_message_proto_msgTypes[9].OneofWrappers = []any{ + (*Failure_ApplicationFailureInfo)(nil), + (*Failure_TimeoutFailureInfo)(nil), + (*Failure_CanceledFailureInfo)(nil), + (*Failure_TerminatedFailureInfo)(nil), + (*Failure_ServerFailureInfo)(nil), + (*Failure_ResetWorkflowFailureInfo)(nil), + (*Failure_ActivityFailureInfo)(nil), + (*Failure_ChildWorkflowExecutionFailureInfo)(nil), + (*Failure_NexusOperationExecutionFailureInfo)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_failure_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 11, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_failure_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_failure_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_failure_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_failure_v1_message_proto = out.File + file_temporal_api_failure_v1_message_proto_rawDesc = nil + file_temporal_api_failure_v1_message_proto_goTypes = nil + file_temporal_api_failure_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/filter/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/filter/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..055f7a0bc54 --- /dev/null +++ b/vendor/go.temporal.io/api/filter/v1/message.go-helpers.pb.go @@ -0,0 +1,176 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package filter + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type WorkflowExecutionFilter to the protobuf v3 wire format +func (val *WorkflowExecutionFilter) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionFilter from the protobuf v3 wire format +func (val *WorkflowExecutionFilter) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionFilter) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionFilter values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionFilter) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionFilter + switch t := that.(type) { + case *WorkflowExecutionFilter: + that1 = t + case WorkflowExecutionFilter: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowTypeFilter to the protobuf v3 wire format +func (val *WorkflowTypeFilter) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowTypeFilter from the protobuf v3 wire format +func (val *WorkflowTypeFilter) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowTypeFilter) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowTypeFilter values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowTypeFilter) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowTypeFilter + switch t := that.(type) { + case *WorkflowTypeFilter: + that1 = t + case WorkflowTypeFilter: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StartTimeFilter to the protobuf v3 wire format +func (val *StartTimeFilter) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StartTimeFilter from the protobuf v3 wire format +func (val *StartTimeFilter) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StartTimeFilter) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StartTimeFilter values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StartTimeFilter) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StartTimeFilter + switch t := that.(type) { + case *StartTimeFilter: + that1 = t + case StartTimeFilter: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StatusFilter to the protobuf v3 wire format +func (val *StatusFilter) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StatusFilter from the protobuf v3 wire format +func (val *StatusFilter) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StatusFilter) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StatusFilter values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StatusFilter) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StatusFilter + switch t := that.(type) { + case *StatusFilter: + that1 = t + case StatusFilter: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/filter/v1/message.pb.go b/vendor/go.temporal.io/api/filter/v1/message.pb.go new file mode 100644 index 00000000000..98f11cfc5d7 --- /dev/null +++ b/vendor/go.temporal.io/api/filter/v1/message.pb.go @@ -0,0 +1,402 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/filter/v1/message.proto + +package filter + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/enums/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type WorkflowExecutionFilter struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + WorkflowId string `protobuf:"bytes,1,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + RunId string `protobuf:"bytes,2,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` +} + +func (x *WorkflowExecutionFilter) Reset() { + *x = WorkflowExecutionFilter{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_filter_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionFilter) ProtoMessage() {} + +func (x *WorkflowExecutionFilter) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_filter_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionFilter.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionFilter) Descriptor() ([]byte, []int) { + return file_temporal_api_filter_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *WorkflowExecutionFilter) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *WorkflowExecutionFilter) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +type WorkflowTypeFilter struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` +} + +func (x *WorkflowTypeFilter) Reset() { + *x = WorkflowTypeFilter{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_filter_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowTypeFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowTypeFilter) ProtoMessage() {} + +func (x *WorkflowTypeFilter) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_filter_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowTypeFilter.ProtoReflect.Descriptor instead. +func (*WorkflowTypeFilter) Descriptor() ([]byte, []int) { + return file_temporal_api_filter_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *WorkflowTypeFilter) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type StartTimeFilter struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + EarliestTime *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=earliest_time,json=earliestTime,proto3" json:"earliest_time,omitempty"` + LatestTime *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=latest_time,json=latestTime,proto3" json:"latest_time,omitempty"` +} + +func (x *StartTimeFilter) Reset() { + *x = StartTimeFilter{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_filter_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartTimeFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartTimeFilter) ProtoMessage() {} + +func (x *StartTimeFilter) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_filter_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartTimeFilter.ProtoReflect.Descriptor instead. +func (*StartTimeFilter) Descriptor() ([]byte, []int) { + return file_temporal_api_filter_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *StartTimeFilter) GetEarliestTime() *timestamppb.Timestamp { + if x != nil { + return x.EarliestTime + } + return nil +} + +func (x *StartTimeFilter) GetLatestTime() *timestamppb.Timestamp { + if x != nil { + return x.LatestTime + } + return nil +} + +type StatusFilter struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status v1.WorkflowExecutionStatus `protobuf:"varint,1,opt,name=status,proto3,enum=temporal.api.enums.v1.WorkflowExecutionStatus" json:"status,omitempty"` +} + +func (x *StatusFilter) Reset() { + *x = StatusFilter{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_filter_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StatusFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StatusFilter) ProtoMessage() {} + +func (x *StatusFilter) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_filter_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StatusFilter.ProtoReflect.Descriptor instead. +func (*StatusFilter) Descriptor() ([]byte, []int) { + return file_temporal_api_filter_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *StatusFilter) GetStatus() v1.WorkflowExecutionStatus { + if x != nil { + return x.Status + } + return v1.WorkflowExecutionStatus(0) +} + +var File_temporal_api_filter_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_filter_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x66, 0x69, + 0x6c, 0x74, 0x65, 0x72, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x16, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, + 0x31, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x22, 0x59, 0x0a, 0x17, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x19, 0x0a, + 0x06, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x72, + 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x2c, 0x0a, 0x12, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x16, 0x0a, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x22, 0x97, 0x01, 0x0a, 0x0f, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, + 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x43, 0x0a, 0x0d, 0x65, 0x61, 0x72, 0x6c, 0x69, + 0x65, 0x73, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0c, 0x65, 0x61, 0x72, 0x6c, 0x69, 0x65, + 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, 0x0a, 0x0b, 0x6c, 0x61, 0x74, + 0x65, 0x73, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0a, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, + 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x5a, 0x0a, 0x0c, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x4a, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x42, 0x89, 0x01, 0x0a, 0x19, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x42, 0x0c, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x23, 0x67, + 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, + 0x2f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2f, 0x76, 0x31, 0x3b, 0x66, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0xaa, 0x02, 0x18, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, + 0x69, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1b, 0x54, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x46, 0x69, + 0x6c, 0x74, 0x65, 0x72, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_filter_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_filter_v1_message_proto_rawDescData = file_temporal_api_filter_v1_message_proto_rawDesc +) + +func file_temporal_api_filter_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_filter_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_filter_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_filter_v1_message_proto_rawDescData) + }) + return file_temporal_api_filter_v1_message_proto_rawDescData +} + +var file_temporal_api_filter_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 4) +var file_temporal_api_filter_v1_message_proto_goTypes = []any{ + (*WorkflowExecutionFilter)(nil), // 0: temporal.api.filter.v1.WorkflowExecutionFilter + (*WorkflowTypeFilter)(nil), // 1: temporal.api.filter.v1.WorkflowTypeFilter + (*StartTimeFilter)(nil), // 2: temporal.api.filter.v1.StartTimeFilter + (*StatusFilter)(nil), // 3: temporal.api.filter.v1.StatusFilter + (*timestamppb.Timestamp)(nil), // 4: google.protobuf.Timestamp + (v1.WorkflowExecutionStatus)(0), // 5: temporal.api.enums.v1.WorkflowExecutionStatus +} +var file_temporal_api_filter_v1_message_proto_depIdxs = []int32{ + 4, // 0: temporal.api.filter.v1.StartTimeFilter.earliest_time:type_name -> google.protobuf.Timestamp + 4, // 1: temporal.api.filter.v1.StartTimeFilter.latest_time:type_name -> google.protobuf.Timestamp + 5, // 2: temporal.api.filter.v1.StatusFilter.status:type_name -> temporal.api.enums.v1.WorkflowExecutionStatus + 3, // [3:3] is the sub-list for method output_type + 3, // [3:3] is the sub-list for method input_type + 3, // [3:3] is the sub-list for extension type_name + 3, // [3:3] is the sub-list for extension extendee + 0, // [0:3] is the sub-list for field type_name +} + +func init() { file_temporal_api_filter_v1_message_proto_init() } +func file_temporal_api_filter_v1_message_proto_init() { + if File_temporal_api_filter_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_filter_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionFilter); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_filter_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowTypeFilter); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_filter_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*StartTimeFilter); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_filter_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*StatusFilter); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_filter_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 4, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_filter_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_filter_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_filter_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_filter_v1_message_proto = out.File + file_temporal_api_filter_v1_message_proto_rawDesc = nil + file_temporal_api_filter_v1_message_proto_goTypes = nil + file_temporal_api_filter_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/history/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/history/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..1572f71ed77 --- /dev/null +++ b/vendor/go.temporal.io/api/history/v1/message.go-helpers.pb.go @@ -0,0 +1,2100 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package history + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type WorkflowExecutionStartedEventAttributes to the protobuf v3 wire format +func (val *WorkflowExecutionStartedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionStartedEventAttributes from the protobuf v3 wire format +func (val *WorkflowExecutionStartedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionStartedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionStartedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionStartedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionStartedEventAttributes + switch t := that.(type) { + case *WorkflowExecutionStartedEventAttributes: + that1 = t + case WorkflowExecutionStartedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionCompletedEventAttributes to the protobuf v3 wire format +func (val *WorkflowExecutionCompletedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionCompletedEventAttributes from the protobuf v3 wire format +func (val *WorkflowExecutionCompletedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionCompletedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionCompletedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionCompletedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionCompletedEventAttributes + switch t := that.(type) { + case *WorkflowExecutionCompletedEventAttributes: + that1 = t + case WorkflowExecutionCompletedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionFailedEventAttributes to the protobuf v3 wire format +func (val *WorkflowExecutionFailedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionFailedEventAttributes from the protobuf v3 wire format +func (val *WorkflowExecutionFailedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionFailedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionFailedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionFailedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionFailedEventAttributes + switch t := that.(type) { + case *WorkflowExecutionFailedEventAttributes: + that1 = t + case WorkflowExecutionFailedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionTimedOutEventAttributes to the protobuf v3 wire format +func (val *WorkflowExecutionTimedOutEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionTimedOutEventAttributes from the protobuf v3 wire format +func (val *WorkflowExecutionTimedOutEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionTimedOutEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionTimedOutEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionTimedOutEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionTimedOutEventAttributes + switch t := that.(type) { + case *WorkflowExecutionTimedOutEventAttributes: + that1 = t + case WorkflowExecutionTimedOutEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionContinuedAsNewEventAttributes to the protobuf v3 wire format +func (val *WorkflowExecutionContinuedAsNewEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionContinuedAsNewEventAttributes from the protobuf v3 wire format +func (val *WorkflowExecutionContinuedAsNewEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionContinuedAsNewEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionContinuedAsNewEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionContinuedAsNewEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionContinuedAsNewEventAttributes + switch t := that.(type) { + case *WorkflowExecutionContinuedAsNewEventAttributes: + that1 = t + case WorkflowExecutionContinuedAsNewEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowTaskScheduledEventAttributes to the protobuf v3 wire format +func (val *WorkflowTaskScheduledEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowTaskScheduledEventAttributes from the protobuf v3 wire format +func (val *WorkflowTaskScheduledEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowTaskScheduledEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowTaskScheduledEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowTaskScheduledEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowTaskScheduledEventAttributes + switch t := that.(type) { + case *WorkflowTaskScheduledEventAttributes: + that1 = t + case WorkflowTaskScheduledEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowTaskStartedEventAttributes to the protobuf v3 wire format +func (val *WorkflowTaskStartedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowTaskStartedEventAttributes from the protobuf v3 wire format +func (val *WorkflowTaskStartedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowTaskStartedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowTaskStartedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowTaskStartedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowTaskStartedEventAttributes + switch t := that.(type) { + case *WorkflowTaskStartedEventAttributes: + that1 = t + case WorkflowTaskStartedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowTaskCompletedEventAttributes to the protobuf v3 wire format +func (val *WorkflowTaskCompletedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowTaskCompletedEventAttributes from the protobuf v3 wire format +func (val *WorkflowTaskCompletedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowTaskCompletedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowTaskCompletedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowTaskCompletedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowTaskCompletedEventAttributes + switch t := that.(type) { + case *WorkflowTaskCompletedEventAttributes: + that1 = t + case WorkflowTaskCompletedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowTaskTimedOutEventAttributes to the protobuf v3 wire format +func (val *WorkflowTaskTimedOutEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowTaskTimedOutEventAttributes from the protobuf v3 wire format +func (val *WorkflowTaskTimedOutEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowTaskTimedOutEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowTaskTimedOutEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowTaskTimedOutEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowTaskTimedOutEventAttributes + switch t := that.(type) { + case *WorkflowTaskTimedOutEventAttributes: + that1 = t + case WorkflowTaskTimedOutEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowTaskFailedEventAttributes to the protobuf v3 wire format +func (val *WorkflowTaskFailedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowTaskFailedEventAttributes from the protobuf v3 wire format +func (val *WorkflowTaskFailedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowTaskFailedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowTaskFailedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowTaskFailedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowTaskFailedEventAttributes + switch t := that.(type) { + case *WorkflowTaskFailedEventAttributes: + that1 = t + case WorkflowTaskFailedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ActivityTaskScheduledEventAttributes to the protobuf v3 wire format +func (val *ActivityTaskScheduledEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ActivityTaskScheduledEventAttributes from the protobuf v3 wire format +func (val *ActivityTaskScheduledEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ActivityTaskScheduledEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ActivityTaskScheduledEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ActivityTaskScheduledEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ActivityTaskScheduledEventAttributes + switch t := that.(type) { + case *ActivityTaskScheduledEventAttributes: + that1 = t + case ActivityTaskScheduledEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ActivityTaskStartedEventAttributes to the protobuf v3 wire format +func (val *ActivityTaskStartedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ActivityTaskStartedEventAttributes from the protobuf v3 wire format +func (val *ActivityTaskStartedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ActivityTaskStartedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ActivityTaskStartedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ActivityTaskStartedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ActivityTaskStartedEventAttributes + switch t := that.(type) { + case *ActivityTaskStartedEventAttributes: + that1 = t + case ActivityTaskStartedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ActivityTaskCompletedEventAttributes to the protobuf v3 wire format +func (val *ActivityTaskCompletedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ActivityTaskCompletedEventAttributes from the protobuf v3 wire format +func (val *ActivityTaskCompletedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ActivityTaskCompletedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ActivityTaskCompletedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ActivityTaskCompletedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ActivityTaskCompletedEventAttributes + switch t := that.(type) { + case *ActivityTaskCompletedEventAttributes: + that1 = t + case ActivityTaskCompletedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ActivityTaskFailedEventAttributes to the protobuf v3 wire format +func (val *ActivityTaskFailedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ActivityTaskFailedEventAttributes from the protobuf v3 wire format +func (val *ActivityTaskFailedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ActivityTaskFailedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ActivityTaskFailedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ActivityTaskFailedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ActivityTaskFailedEventAttributes + switch t := that.(type) { + case *ActivityTaskFailedEventAttributes: + that1 = t + case ActivityTaskFailedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ActivityTaskTimedOutEventAttributes to the protobuf v3 wire format +func (val *ActivityTaskTimedOutEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ActivityTaskTimedOutEventAttributes from the protobuf v3 wire format +func (val *ActivityTaskTimedOutEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ActivityTaskTimedOutEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ActivityTaskTimedOutEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ActivityTaskTimedOutEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ActivityTaskTimedOutEventAttributes + switch t := that.(type) { + case *ActivityTaskTimedOutEventAttributes: + that1 = t + case ActivityTaskTimedOutEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ActivityTaskCancelRequestedEventAttributes to the protobuf v3 wire format +func (val *ActivityTaskCancelRequestedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ActivityTaskCancelRequestedEventAttributes from the protobuf v3 wire format +func (val *ActivityTaskCancelRequestedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ActivityTaskCancelRequestedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ActivityTaskCancelRequestedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ActivityTaskCancelRequestedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ActivityTaskCancelRequestedEventAttributes + switch t := that.(type) { + case *ActivityTaskCancelRequestedEventAttributes: + that1 = t + case ActivityTaskCancelRequestedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ActivityTaskCanceledEventAttributes to the protobuf v3 wire format +func (val *ActivityTaskCanceledEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ActivityTaskCanceledEventAttributes from the protobuf v3 wire format +func (val *ActivityTaskCanceledEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ActivityTaskCanceledEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ActivityTaskCanceledEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ActivityTaskCanceledEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ActivityTaskCanceledEventAttributes + switch t := that.(type) { + case *ActivityTaskCanceledEventAttributes: + that1 = t + case ActivityTaskCanceledEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TimerStartedEventAttributes to the protobuf v3 wire format +func (val *TimerStartedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TimerStartedEventAttributes from the protobuf v3 wire format +func (val *TimerStartedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TimerStartedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TimerStartedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TimerStartedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TimerStartedEventAttributes + switch t := that.(type) { + case *TimerStartedEventAttributes: + that1 = t + case TimerStartedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TimerFiredEventAttributes to the protobuf v3 wire format +func (val *TimerFiredEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TimerFiredEventAttributes from the protobuf v3 wire format +func (val *TimerFiredEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TimerFiredEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TimerFiredEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TimerFiredEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TimerFiredEventAttributes + switch t := that.(type) { + case *TimerFiredEventAttributes: + that1 = t + case TimerFiredEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TimerCanceledEventAttributes to the protobuf v3 wire format +func (val *TimerCanceledEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TimerCanceledEventAttributes from the protobuf v3 wire format +func (val *TimerCanceledEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TimerCanceledEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TimerCanceledEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TimerCanceledEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TimerCanceledEventAttributes + switch t := that.(type) { + case *TimerCanceledEventAttributes: + that1 = t + case TimerCanceledEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionCancelRequestedEventAttributes to the protobuf v3 wire format +func (val *WorkflowExecutionCancelRequestedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionCancelRequestedEventAttributes from the protobuf v3 wire format +func (val *WorkflowExecutionCancelRequestedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionCancelRequestedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionCancelRequestedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionCancelRequestedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionCancelRequestedEventAttributes + switch t := that.(type) { + case *WorkflowExecutionCancelRequestedEventAttributes: + that1 = t + case WorkflowExecutionCancelRequestedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionCanceledEventAttributes to the protobuf v3 wire format +func (val *WorkflowExecutionCanceledEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionCanceledEventAttributes from the protobuf v3 wire format +func (val *WorkflowExecutionCanceledEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionCanceledEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionCanceledEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionCanceledEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionCanceledEventAttributes + switch t := that.(type) { + case *WorkflowExecutionCanceledEventAttributes: + that1 = t + case WorkflowExecutionCanceledEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type MarkerRecordedEventAttributes to the protobuf v3 wire format +func (val *MarkerRecordedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type MarkerRecordedEventAttributes from the protobuf v3 wire format +func (val *MarkerRecordedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *MarkerRecordedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two MarkerRecordedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *MarkerRecordedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *MarkerRecordedEventAttributes + switch t := that.(type) { + case *MarkerRecordedEventAttributes: + that1 = t + case MarkerRecordedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionSignaledEventAttributes to the protobuf v3 wire format +func (val *WorkflowExecutionSignaledEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionSignaledEventAttributes from the protobuf v3 wire format +func (val *WorkflowExecutionSignaledEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionSignaledEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionSignaledEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionSignaledEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionSignaledEventAttributes + switch t := that.(type) { + case *WorkflowExecutionSignaledEventAttributes: + that1 = t + case WorkflowExecutionSignaledEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionTerminatedEventAttributes to the protobuf v3 wire format +func (val *WorkflowExecutionTerminatedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionTerminatedEventAttributes from the protobuf v3 wire format +func (val *WorkflowExecutionTerminatedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionTerminatedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionTerminatedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionTerminatedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionTerminatedEventAttributes + switch t := that.(type) { + case *WorkflowExecutionTerminatedEventAttributes: + that1 = t + case WorkflowExecutionTerminatedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RequestCancelExternalWorkflowExecutionInitiatedEventAttributes to the protobuf v3 wire format +func (val *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RequestCancelExternalWorkflowExecutionInitiatedEventAttributes from the protobuf v3 wire format +func (val *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RequestCancelExternalWorkflowExecutionInitiatedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes + switch t := that.(type) { + case *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes: + that1 = t + case RequestCancelExternalWorkflowExecutionInitiatedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RequestCancelExternalWorkflowExecutionFailedEventAttributes to the protobuf v3 wire format +func (val *RequestCancelExternalWorkflowExecutionFailedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RequestCancelExternalWorkflowExecutionFailedEventAttributes from the protobuf v3 wire format +func (val *RequestCancelExternalWorkflowExecutionFailedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RequestCancelExternalWorkflowExecutionFailedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RequestCancelExternalWorkflowExecutionFailedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RequestCancelExternalWorkflowExecutionFailedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RequestCancelExternalWorkflowExecutionFailedEventAttributes + switch t := that.(type) { + case *RequestCancelExternalWorkflowExecutionFailedEventAttributes: + that1 = t + case RequestCancelExternalWorkflowExecutionFailedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ExternalWorkflowExecutionCancelRequestedEventAttributes to the protobuf v3 wire format +func (val *ExternalWorkflowExecutionCancelRequestedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ExternalWorkflowExecutionCancelRequestedEventAttributes from the protobuf v3 wire format +func (val *ExternalWorkflowExecutionCancelRequestedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ExternalWorkflowExecutionCancelRequestedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ExternalWorkflowExecutionCancelRequestedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ExternalWorkflowExecutionCancelRequestedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ExternalWorkflowExecutionCancelRequestedEventAttributes + switch t := that.(type) { + case *ExternalWorkflowExecutionCancelRequestedEventAttributes: + that1 = t + case ExternalWorkflowExecutionCancelRequestedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SignalExternalWorkflowExecutionInitiatedEventAttributes to the protobuf v3 wire format +func (val *SignalExternalWorkflowExecutionInitiatedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SignalExternalWorkflowExecutionInitiatedEventAttributes from the protobuf v3 wire format +func (val *SignalExternalWorkflowExecutionInitiatedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SignalExternalWorkflowExecutionInitiatedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SignalExternalWorkflowExecutionInitiatedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SignalExternalWorkflowExecutionInitiatedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SignalExternalWorkflowExecutionInitiatedEventAttributes + switch t := that.(type) { + case *SignalExternalWorkflowExecutionInitiatedEventAttributes: + that1 = t + case SignalExternalWorkflowExecutionInitiatedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SignalExternalWorkflowExecutionFailedEventAttributes to the protobuf v3 wire format +func (val *SignalExternalWorkflowExecutionFailedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SignalExternalWorkflowExecutionFailedEventAttributes from the protobuf v3 wire format +func (val *SignalExternalWorkflowExecutionFailedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SignalExternalWorkflowExecutionFailedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SignalExternalWorkflowExecutionFailedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SignalExternalWorkflowExecutionFailedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SignalExternalWorkflowExecutionFailedEventAttributes + switch t := that.(type) { + case *SignalExternalWorkflowExecutionFailedEventAttributes: + that1 = t + case SignalExternalWorkflowExecutionFailedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ExternalWorkflowExecutionSignaledEventAttributes to the protobuf v3 wire format +func (val *ExternalWorkflowExecutionSignaledEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ExternalWorkflowExecutionSignaledEventAttributes from the protobuf v3 wire format +func (val *ExternalWorkflowExecutionSignaledEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ExternalWorkflowExecutionSignaledEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ExternalWorkflowExecutionSignaledEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ExternalWorkflowExecutionSignaledEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ExternalWorkflowExecutionSignaledEventAttributes + switch t := that.(type) { + case *ExternalWorkflowExecutionSignaledEventAttributes: + that1 = t + case ExternalWorkflowExecutionSignaledEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpsertWorkflowSearchAttributesEventAttributes to the protobuf v3 wire format +func (val *UpsertWorkflowSearchAttributesEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpsertWorkflowSearchAttributesEventAttributes from the protobuf v3 wire format +func (val *UpsertWorkflowSearchAttributesEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpsertWorkflowSearchAttributesEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpsertWorkflowSearchAttributesEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpsertWorkflowSearchAttributesEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpsertWorkflowSearchAttributesEventAttributes + switch t := that.(type) { + case *UpsertWorkflowSearchAttributesEventAttributes: + that1 = t + case UpsertWorkflowSearchAttributesEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowPropertiesModifiedEventAttributes to the protobuf v3 wire format +func (val *WorkflowPropertiesModifiedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowPropertiesModifiedEventAttributes from the protobuf v3 wire format +func (val *WorkflowPropertiesModifiedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowPropertiesModifiedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowPropertiesModifiedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowPropertiesModifiedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowPropertiesModifiedEventAttributes + switch t := that.(type) { + case *WorkflowPropertiesModifiedEventAttributes: + that1 = t + case WorkflowPropertiesModifiedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StartChildWorkflowExecutionInitiatedEventAttributes to the protobuf v3 wire format +func (val *StartChildWorkflowExecutionInitiatedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StartChildWorkflowExecutionInitiatedEventAttributes from the protobuf v3 wire format +func (val *StartChildWorkflowExecutionInitiatedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StartChildWorkflowExecutionInitiatedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StartChildWorkflowExecutionInitiatedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StartChildWorkflowExecutionInitiatedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StartChildWorkflowExecutionInitiatedEventAttributes + switch t := that.(type) { + case *StartChildWorkflowExecutionInitiatedEventAttributes: + that1 = t + case StartChildWorkflowExecutionInitiatedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StartChildWorkflowExecutionFailedEventAttributes to the protobuf v3 wire format +func (val *StartChildWorkflowExecutionFailedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StartChildWorkflowExecutionFailedEventAttributes from the protobuf v3 wire format +func (val *StartChildWorkflowExecutionFailedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StartChildWorkflowExecutionFailedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StartChildWorkflowExecutionFailedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StartChildWorkflowExecutionFailedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StartChildWorkflowExecutionFailedEventAttributes + switch t := that.(type) { + case *StartChildWorkflowExecutionFailedEventAttributes: + that1 = t + case StartChildWorkflowExecutionFailedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ChildWorkflowExecutionStartedEventAttributes to the protobuf v3 wire format +func (val *ChildWorkflowExecutionStartedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ChildWorkflowExecutionStartedEventAttributes from the protobuf v3 wire format +func (val *ChildWorkflowExecutionStartedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ChildWorkflowExecutionStartedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ChildWorkflowExecutionStartedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ChildWorkflowExecutionStartedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ChildWorkflowExecutionStartedEventAttributes + switch t := that.(type) { + case *ChildWorkflowExecutionStartedEventAttributes: + that1 = t + case ChildWorkflowExecutionStartedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ChildWorkflowExecutionCompletedEventAttributes to the protobuf v3 wire format +func (val *ChildWorkflowExecutionCompletedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ChildWorkflowExecutionCompletedEventAttributes from the protobuf v3 wire format +func (val *ChildWorkflowExecutionCompletedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ChildWorkflowExecutionCompletedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ChildWorkflowExecutionCompletedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ChildWorkflowExecutionCompletedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ChildWorkflowExecutionCompletedEventAttributes + switch t := that.(type) { + case *ChildWorkflowExecutionCompletedEventAttributes: + that1 = t + case ChildWorkflowExecutionCompletedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ChildWorkflowExecutionFailedEventAttributes to the protobuf v3 wire format +func (val *ChildWorkflowExecutionFailedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ChildWorkflowExecutionFailedEventAttributes from the protobuf v3 wire format +func (val *ChildWorkflowExecutionFailedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ChildWorkflowExecutionFailedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ChildWorkflowExecutionFailedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ChildWorkflowExecutionFailedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ChildWorkflowExecutionFailedEventAttributes + switch t := that.(type) { + case *ChildWorkflowExecutionFailedEventAttributes: + that1 = t + case ChildWorkflowExecutionFailedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ChildWorkflowExecutionCanceledEventAttributes to the protobuf v3 wire format +func (val *ChildWorkflowExecutionCanceledEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ChildWorkflowExecutionCanceledEventAttributes from the protobuf v3 wire format +func (val *ChildWorkflowExecutionCanceledEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ChildWorkflowExecutionCanceledEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ChildWorkflowExecutionCanceledEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ChildWorkflowExecutionCanceledEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ChildWorkflowExecutionCanceledEventAttributes + switch t := that.(type) { + case *ChildWorkflowExecutionCanceledEventAttributes: + that1 = t + case ChildWorkflowExecutionCanceledEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ChildWorkflowExecutionTimedOutEventAttributes to the protobuf v3 wire format +func (val *ChildWorkflowExecutionTimedOutEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ChildWorkflowExecutionTimedOutEventAttributes from the protobuf v3 wire format +func (val *ChildWorkflowExecutionTimedOutEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ChildWorkflowExecutionTimedOutEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ChildWorkflowExecutionTimedOutEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ChildWorkflowExecutionTimedOutEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ChildWorkflowExecutionTimedOutEventAttributes + switch t := that.(type) { + case *ChildWorkflowExecutionTimedOutEventAttributes: + that1 = t + case ChildWorkflowExecutionTimedOutEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ChildWorkflowExecutionTerminatedEventAttributes to the protobuf v3 wire format +func (val *ChildWorkflowExecutionTerminatedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ChildWorkflowExecutionTerminatedEventAttributes from the protobuf v3 wire format +func (val *ChildWorkflowExecutionTerminatedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ChildWorkflowExecutionTerminatedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ChildWorkflowExecutionTerminatedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ChildWorkflowExecutionTerminatedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ChildWorkflowExecutionTerminatedEventAttributes + switch t := that.(type) { + case *ChildWorkflowExecutionTerminatedEventAttributes: + that1 = t + case ChildWorkflowExecutionTerminatedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowPropertiesModifiedExternallyEventAttributes to the protobuf v3 wire format +func (val *WorkflowPropertiesModifiedExternallyEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowPropertiesModifiedExternallyEventAttributes from the protobuf v3 wire format +func (val *WorkflowPropertiesModifiedExternallyEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowPropertiesModifiedExternallyEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowPropertiesModifiedExternallyEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowPropertiesModifiedExternallyEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowPropertiesModifiedExternallyEventAttributes + switch t := that.(type) { + case *WorkflowPropertiesModifiedExternallyEventAttributes: + that1 = t + case WorkflowPropertiesModifiedExternallyEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ActivityPropertiesModifiedExternallyEventAttributes to the protobuf v3 wire format +func (val *ActivityPropertiesModifiedExternallyEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ActivityPropertiesModifiedExternallyEventAttributes from the protobuf v3 wire format +func (val *ActivityPropertiesModifiedExternallyEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ActivityPropertiesModifiedExternallyEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ActivityPropertiesModifiedExternallyEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ActivityPropertiesModifiedExternallyEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ActivityPropertiesModifiedExternallyEventAttributes + switch t := that.(type) { + case *ActivityPropertiesModifiedExternallyEventAttributes: + that1 = t + case ActivityPropertiesModifiedExternallyEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionUpdateAcceptedEventAttributes to the protobuf v3 wire format +func (val *WorkflowExecutionUpdateAcceptedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionUpdateAcceptedEventAttributes from the protobuf v3 wire format +func (val *WorkflowExecutionUpdateAcceptedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionUpdateAcceptedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionUpdateAcceptedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionUpdateAcceptedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionUpdateAcceptedEventAttributes + switch t := that.(type) { + case *WorkflowExecutionUpdateAcceptedEventAttributes: + that1 = t + case WorkflowExecutionUpdateAcceptedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionUpdateCompletedEventAttributes to the protobuf v3 wire format +func (val *WorkflowExecutionUpdateCompletedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionUpdateCompletedEventAttributes from the protobuf v3 wire format +func (val *WorkflowExecutionUpdateCompletedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionUpdateCompletedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionUpdateCompletedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionUpdateCompletedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionUpdateCompletedEventAttributes + switch t := that.(type) { + case *WorkflowExecutionUpdateCompletedEventAttributes: + that1 = t + case WorkflowExecutionUpdateCompletedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionUpdateRejectedEventAttributes to the protobuf v3 wire format +func (val *WorkflowExecutionUpdateRejectedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionUpdateRejectedEventAttributes from the protobuf v3 wire format +func (val *WorkflowExecutionUpdateRejectedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionUpdateRejectedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionUpdateRejectedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionUpdateRejectedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionUpdateRejectedEventAttributes + switch t := that.(type) { + case *WorkflowExecutionUpdateRejectedEventAttributes: + that1 = t + case WorkflowExecutionUpdateRejectedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionUpdateAdmittedEventAttributes to the protobuf v3 wire format +func (val *WorkflowExecutionUpdateAdmittedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionUpdateAdmittedEventAttributes from the protobuf v3 wire format +func (val *WorkflowExecutionUpdateAdmittedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionUpdateAdmittedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionUpdateAdmittedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionUpdateAdmittedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionUpdateAdmittedEventAttributes + switch t := that.(type) { + case *WorkflowExecutionUpdateAdmittedEventAttributes: + that1 = t + case WorkflowExecutionUpdateAdmittedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NexusOperationScheduledEventAttributes to the protobuf v3 wire format +func (val *NexusOperationScheduledEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NexusOperationScheduledEventAttributes from the protobuf v3 wire format +func (val *NexusOperationScheduledEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NexusOperationScheduledEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NexusOperationScheduledEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NexusOperationScheduledEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NexusOperationScheduledEventAttributes + switch t := that.(type) { + case *NexusOperationScheduledEventAttributes: + that1 = t + case NexusOperationScheduledEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NexusOperationStartedEventAttributes to the protobuf v3 wire format +func (val *NexusOperationStartedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NexusOperationStartedEventAttributes from the protobuf v3 wire format +func (val *NexusOperationStartedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NexusOperationStartedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NexusOperationStartedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NexusOperationStartedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NexusOperationStartedEventAttributes + switch t := that.(type) { + case *NexusOperationStartedEventAttributes: + that1 = t + case NexusOperationStartedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NexusOperationCompletedEventAttributes to the protobuf v3 wire format +func (val *NexusOperationCompletedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NexusOperationCompletedEventAttributes from the protobuf v3 wire format +func (val *NexusOperationCompletedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NexusOperationCompletedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NexusOperationCompletedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NexusOperationCompletedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NexusOperationCompletedEventAttributes + switch t := that.(type) { + case *NexusOperationCompletedEventAttributes: + that1 = t + case NexusOperationCompletedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NexusOperationFailedEventAttributes to the protobuf v3 wire format +func (val *NexusOperationFailedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NexusOperationFailedEventAttributes from the protobuf v3 wire format +func (val *NexusOperationFailedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NexusOperationFailedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NexusOperationFailedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NexusOperationFailedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NexusOperationFailedEventAttributes + switch t := that.(type) { + case *NexusOperationFailedEventAttributes: + that1 = t + case NexusOperationFailedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NexusOperationTimedOutEventAttributes to the protobuf v3 wire format +func (val *NexusOperationTimedOutEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NexusOperationTimedOutEventAttributes from the protobuf v3 wire format +func (val *NexusOperationTimedOutEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NexusOperationTimedOutEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NexusOperationTimedOutEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NexusOperationTimedOutEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NexusOperationTimedOutEventAttributes + switch t := that.(type) { + case *NexusOperationTimedOutEventAttributes: + that1 = t + case NexusOperationTimedOutEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NexusOperationCanceledEventAttributes to the protobuf v3 wire format +func (val *NexusOperationCanceledEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NexusOperationCanceledEventAttributes from the protobuf v3 wire format +func (val *NexusOperationCanceledEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NexusOperationCanceledEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NexusOperationCanceledEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NexusOperationCanceledEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NexusOperationCanceledEventAttributes + switch t := that.(type) { + case *NexusOperationCanceledEventAttributes: + that1 = t + case NexusOperationCanceledEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NexusOperationCancelRequestedEventAttributes to the protobuf v3 wire format +func (val *NexusOperationCancelRequestedEventAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NexusOperationCancelRequestedEventAttributes from the protobuf v3 wire format +func (val *NexusOperationCancelRequestedEventAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NexusOperationCancelRequestedEventAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NexusOperationCancelRequestedEventAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NexusOperationCancelRequestedEventAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NexusOperationCancelRequestedEventAttributes + switch t := that.(type) { + case *NexusOperationCancelRequestedEventAttributes: + that1 = t + case NexusOperationCancelRequestedEventAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type HistoryEvent to the protobuf v3 wire format +func (val *HistoryEvent) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type HistoryEvent from the protobuf v3 wire format +func (val *HistoryEvent) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *HistoryEvent) Size() int { + return proto.Size(val) +} + +// Equal returns whether two HistoryEvent values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *HistoryEvent) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *HistoryEvent + switch t := that.(type) { + case *HistoryEvent: + that1 = t + case HistoryEvent: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type History to the protobuf v3 wire format +func (val *History) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type History from the protobuf v3 wire format +func (val *History) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *History) Size() int { + return proto.Size(val) +} + +// Equal returns whether two History values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *History) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *History + switch t := that.(type) { + case *History: + that1 = t + case History: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/history/v1/message.pb.go b/vendor/go.temporal.io/api/history/v1/message.pb.go new file mode 100644 index 00000000000..7b3da8c83a6 --- /dev/null +++ b/vendor/go.temporal.io/api/history/v1/message.pb.go @@ -0,0 +1,9015 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/history/v1/message.proto + +package history + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/common/v1" + v12 "go.temporal.io/api/enums/v1" + v13 "go.temporal.io/api/failure/v1" + v15 "go.temporal.io/api/sdk/v1" + v11 "go.temporal.io/api/taskqueue/v1" + v16 "go.temporal.io/api/update/v1" + v14 "go.temporal.io/api/workflow/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + durationpb "google.golang.org/protobuf/types/known/durationpb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// Always the first event in workflow history +type WorkflowExecutionStartedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + WorkflowType *v1.WorkflowType `protobuf:"bytes,1,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + // If this workflow is a child, the namespace our parent lives in. + // SDKs and UI tools should use `parent_workflow_namespace` field but server must use `parent_workflow_namespace_id` only. + ParentWorkflowNamespace string `protobuf:"bytes,2,opt,name=parent_workflow_namespace,json=parentWorkflowNamespace,proto3" json:"parent_workflow_namespace,omitempty"` + ParentWorkflowNamespaceId string `protobuf:"bytes,27,opt,name=parent_workflow_namespace_id,json=parentWorkflowNamespaceId,proto3" json:"parent_workflow_namespace_id,omitempty"` + // Contains information about parent workflow execution that initiated the child workflow these attributes belong to. + // If the workflow these attributes belong to is not a child workflow of any other execution, this field will not be populated. + ParentWorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,3,opt,name=parent_workflow_execution,json=parentWorkflowExecution,proto3" json:"parent_workflow_execution,omitempty"` + // EventID of the child execution initiated event in parent workflow + ParentInitiatedEventId int64 `protobuf:"varint,4,opt,name=parent_initiated_event_id,json=parentInitiatedEventId,proto3" json:"parent_initiated_event_id,omitempty"` + TaskQueue *v11.TaskQueue `protobuf:"bytes,5,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + // SDK will deserialize this and provide it as arguments to the workflow function + Input *v1.Payloads `protobuf:"bytes,6,opt,name=input,proto3" json:"input,omitempty"` + // Total workflow execution timeout including retries and continue as new. + WorkflowExecutionTimeout *durationpb.Duration `protobuf:"bytes,7,opt,name=workflow_execution_timeout,json=workflowExecutionTimeout,proto3" json:"workflow_execution_timeout,omitempty"` + // Timeout of a single workflow run. + WorkflowRunTimeout *durationpb.Duration `protobuf:"bytes,8,opt,name=workflow_run_timeout,json=workflowRunTimeout,proto3" json:"workflow_run_timeout,omitempty"` + // Timeout of a single workflow task. + WorkflowTaskTimeout *durationpb.Duration `protobuf:"bytes,9,opt,name=workflow_task_timeout,json=workflowTaskTimeout,proto3" json:"workflow_task_timeout,omitempty"` + // Run id of the previous workflow which continued-as-new or retired or cron executed into this + // workflow. + ContinuedExecutionRunId string `protobuf:"bytes,10,opt,name=continued_execution_run_id,json=continuedExecutionRunId,proto3" json:"continued_execution_run_id,omitempty"` + Initiator v12.ContinueAsNewInitiator `protobuf:"varint,11,opt,name=initiator,proto3,enum=temporal.api.enums.v1.ContinueAsNewInitiator" json:"initiator,omitempty"` + ContinuedFailure *v13.Failure `protobuf:"bytes,12,opt,name=continued_failure,json=continuedFailure,proto3" json:"continued_failure,omitempty"` + LastCompletionResult *v1.Payloads `protobuf:"bytes,13,opt,name=last_completion_result,json=lastCompletionResult,proto3" json:"last_completion_result,omitempty"` + // This is the run id when the WorkflowExecutionStarted event was written. + // A workflow reset changes the execution run_id, but preserves this field. + OriginalExecutionRunId string `protobuf:"bytes,14,opt,name=original_execution_run_id,json=originalExecutionRunId,proto3" json:"original_execution_run_id,omitempty"` + // Identity of the client who requested this execution + Identity string `protobuf:"bytes,15,opt,name=identity,proto3" json:"identity,omitempty"` + // This is the very first runId along the chain of ContinueAsNew, Retry, Cron and Reset. + // Used to identify a chain. + FirstExecutionRunId string `protobuf:"bytes,16,opt,name=first_execution_run_id,json=firstExecutionRunId,proto3" json:"first_execution_run_id,omitempty"` + RetryPolicy *v1.RetryPolicy `protobuf:"bytes,17,opt,name=retry_policy,json=retryPolicy,proto3" json:"retry_policy,omitempty"` + // Starting at 1, the number of times we have tried to execute this workflow + Attempt int32 `protobuf:"varint,18,opt,name=attempt,proto3" json:"attempt,omitempty"` + // The absolute time at which the workflow will be timed out. + // This is passed without change to the next run/retry of a workflow. + WorkflowExecutionExpirationTime *timestamppb.Timestamp `protobuf:"bytes,19,opt,name=workflow_execution_expiration_time,json=workflowExecutionExpirationTime,proto3" json:"workflow_execution_expiration_time,omitempty"` + // If this workflow runs on a cron schedule, it will appear here + CronSchedule string `protobuf:"bytes,20,opt,name=cron_schedule,json=cronSchedule,proto3" json:"cron_schedule,omitempty"` + // For a cron workflow, this contains the amount of time between when this iteration of + // the cron workflow was scheduled and when it should run next per its cron_schedule. + FirstWorkflowTaskBackoff *durationpb.Duration `protobuf:"bytes,21,opt,name=first_workflow_task_backoff,json=firstWorkflowTaskBackoff,proto3" json:"first_workflow_task_backoff,omitempty"` + Memo *v1.Memo `protobuf:"bytes,22,opt,name=memo,proto3" json:"memo,omitempty"` + SearchAttributes *v1.SearchAttributes `protobuf:"bytes,23,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` + PrevAutoResetPoints *v14.ResetPoints `protobuf:"bytes,24,opt,name=prev_auto_reset_points,json=prevAutoResetPoints,proto3" json:"prev_auto_reset_points,omitempty"` + Header *v1.Header `protobuf:"bytes,25,opt,name=header,proto3" json:"header,omitempty"` + // Version of the child execution initiated event in parent workflow + // It should be used together with parent_initiated_event_id to identify + // a child initiated event for global namespace + ParentInitiatedEventVersion int64 `protobuf:"varint,26,opt,name=parent_initiated_event_version,json=parentInitiatedEventVersion,proto3" json:"parent_initiated_event_version,omitempty"` + // This field is new in 1.21. + WorkflowId string `protobuf:"bytes,28,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + // If this workflow intends to use anything other than the current overall default version for + // the queue, then we include it here. + // Deprecated. use `inherited_build_id` instead + SourceVersionStamp *v1.WorkerVersionStamp `protobuf:"bytes,29,opt,name=source_version_stamp,json=sourceVersionStamp,proto3" json:"source_version_stamp,omitempty"` + // Completion callbacks attached when this workflow was started. + CompletionCallbacks []*v1.Callback `protobuf:"bytes,30,rep,name=completion_callbacks,json=completionCallbacks,proto3" json:"completion_callbacks,omitempty"` + // Contains information about the root workflow execution. + // The root workflow execution is defined as follows: + // 1. A workflow without parent workflow is its own root workflow. + // 2. A workflow that has a parent workflow has the same root workflow as its parent workflow. + // Note: workflows continued as new or reseted may or may not have parents, check examples below. + // + // Examples: + // + // Scenario 1: Workflow W1 starts child workflow W2, and W2 starts child workflow W3. + // - The root workflow of all three workflows is W1. + // Scenario 2: Workflow W1 starts child workflow W2, and W2 continued as new W3. + // - The root workflow of all three workflows is W1. + // Scenario 3: Workflow W1 continued as new W2. + // - The root workflow of W1 is W1 and the root workflow of W2 is W2. + // Scenario 4: Workflow W1 starts child workflow W2, and W2 is reseted, creating W3 + // - The root workflow of all three workflows is W1. + // Scenario 5: Workflow W1 is reseted, creating W2. + // - The root workflow of W1 is W1 and the root workflow of W2 is W2. + RootWorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,31,opt,name=root_workflow_execution,json=rootWorkflowExecution,proto3" json:"root_workflow_execution,omitempty"` + // When present, this execution is assigned to the build ID of its parent or previous execution. + InheritedBuildId string `protobuf:"bytes,32,opt,name=inherited_build_id,json=inheritedBuildId,proto3" json:"inherited_build_id,omitempty"` +} + +func (x *WorkflowExecutionStartedEventAttributes) Reset() { + *x = WorkflowExecutionStartedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionStartedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionStartedEventAttributes) ProtoMessage() {} + +func (x *WorkflowExecutionStartedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionStartedEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionStartedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *WorkflowExecutionStartedEventAttributes) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetParentWorkflowNamespace() string { + if x != nil { + return x.ParentWorkflowNamespace + } + return "" +} + +func (x *WorkflowExecutionStartedEventAttributes) GetParentWorkflowNamespaceId() string { + if x != nil { + return x.ParentWorkflowNamespaceId + } + return "" +} + +func (x *WorkflowExecutionStartedEventAttributes) GetParentWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.ParentWorkflowExecution + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetParentInitiatedEventId() int64 { + if x != nil { + return x.ParentInitiatedEventId + } + return 0 +} + +func (x *WorkflowExecutionStartedEventAttributes) GetTaskQueue() *v11.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetInput() *v1.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetWorkflowExecutionTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowExecutionTimeout + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetWorkflowRunTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowRunTimeout + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetWorkflowTaskTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowTaskTimeout + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetContinuedExecutionRunId() string { + if x != nil { + return x.ContinuedExecutionRunId + } + return "" +} + +func (x *WorkflowExecutionStartedEventAttributes) GetInitiator() v12.ContinueAsNewInitiator { + if x != nil { + return x.Initiator + } + return v12.ContinueAsNewInitiator(0) +} + +func (x *WorkflowExecutionStartedEventAttributes) GetContinuedFailure() *v13.Failure { + if x != nil { + return x.ContinuedFailure + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetLastCompletionResult() *v1.Payloads { + if x != nil { + return x.LastCompletionResult + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetOriginalExecutionRunId() string { + if x != nil { + return x.OriginalExecutionRunId + } + return "" +} + +func (x *WorkflowExecutionStartedEventAttributes) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *WorkflowExecutionStartedEventAttributes) GetFirstExecutionRunId() string { + if x != nil { + return x.FirstExecutionRunId + } + return "" +} + +func (x *WorkflowExecutionStartedEventAttributes) GetRetryPolicy() *v1.RetryPolicy { + if x != nil { + return x.RetryPolicy + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetAttempt() int32 { + if x != nil { + return x.Attempt + } + return 0 +} + +func (x *WorkflowExecutionStartedEventAttributes) GetWorkflowExecutionExpirationTime() *timestamppb.Timestamp { + if x != nil { + return x.WorkflowExecutionExpirationTime + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetCronSchedule() string { + if x != nil { + return x.CronSchedule + } + return "" +} + +func (x *WorkflowExecutionStartedEventAttributes) GetFirstWorkflowTaskBackoff() *durationpb.Duration { + if x != nil { + return x.FirstWorkflowTaskBackoff + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetMemo() *v1.Memo { + if x != nil { + return x.Memo + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetSearchAttributes() *v1.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetPrevAutoResetPoints() *v14.ResetPoints { + if x != nil { + return x.PrevAutoResetPoints + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetParentInitiatedEventVersion() int64 { + if x != nil { + return x.ParentInitiatedEventVersion + } + return 0 +} + +func (x *WorkflowExecutionStartedEventAttributes) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *WorkflowExecutionStartedEventAttributes) GetSourceVersionStamp() *v1.WorkerVersionStamp { + if x != nil { + return x.SourceVersionStamp + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetCompletionCallbacks() []*v1.Callback { + if x != nil { + return x.CompletionCallbacks + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetRootWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.RootWorkflowExecution + } + return nil +} + +func (x *WorkflowExecutionStartedEventAttributes) GetInheritedBuildId() string { + if x != nil { + return x.InheritedBuildId + } + return "" +} + +type WorkflowExecutionCompletedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Serialized result of workflow completion (ie: The return value of the workflow function) + Result *v1.Payloads `protobuf:"bytes,1,opt,name=result,proto3" json:"result,omitempty"` + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,2,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + // If another run is started by cron, this contains the new run id. + NewExecutionRunId string `protobuf:"bytes,3,opt,name=new_execution_run_id,json=newExecutionRunId,proto3" json:"new_execution_run_id,omitempty"` +} + +func (x *WorkflowExecutionCompletedEventAttributes) Reset() { + *x = WorkflowExecutionCompletedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionCompletedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionCompletedEventAttributes) ProtoMessage() {} + +func (x *WorkflowExecutionCompletedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionCompletedEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionCompletedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *WorkflowExecutionCompletedEventAttributes) GetResult() *v1.Payloads { + if x != nil { + return x.Result + } + return nil +} + +func (x *WorkflowExecutionCompletedEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *WorkflowExecutionCompletedEventAttributes) GetNewExecutionRunId() string { + if x != nil { + return x.NewExecutionRunId + } + return "" +} + +type WorkflowExecutionFailedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Serialized result of workflow failure (ex: An exception thrown, or error returned) + Failure *v13.Failure `protobuf:"bytes,1,opt,name=failure,proto3" json:"failure,omitempty"` + RetryState v12.RetryState `protobuf:"varint,2,opt,name=retry_state,json=retryState,proto3,enum=temporal.api.enums.v1.RetryState" json:"retry_state,omitempty"` + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,3,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + // If another run is started by cron or retry, this contains the new run id. + NewExecutionRunId string `protobuf:"bytes,4,opt,name=new_execution_run_id,json=newExecutionRunId,proto3" json:"new_execution_run_id,omitempty"` +} + +func (x *WorkflowExecutionFailedEventAttributes) Reset() { + *x = WorkflowExecutionFailedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionFailedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionFailedEventAttributes) ProtoMessage() {} + +func (x *WorkflowExecutionFailedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionFailedEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionFailedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *WorkflowExecutionFailedEventAttributes) GetFailure() *v13.Failure { + if x != nil { + return x.Failure + } + return nil +} + +func (x *WorkflowExecutionFailedEventAttributes) GetRetryState() v12.RetryState { + if x != nil { + return x.RetryState + } + return v12.RetryState(0) +} + +func (x *WorkflowExecutionFailedEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *WorkflowExecutionFailedEventAttributes) GetNewExecutionRunId() string { + if x != nil { + return x.NewExecutionRunId + } + return "" +} + +type WorkflowExecutionTimedOutEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + RetryState v12.RetryState `protobuf:"varint,1,opt,name=retry_state,json=retryState,proto3,enum=temporal.api.enums.v1.RetryState" json:"retry_state,omitempty"` + // If another run is started by cron or retry, this contains the new run id. + NewExecutionRunId string `protobuf:"bytes,2,opt,name=new_execution_run_id,json=newExecutionRunId,proto3" json:"new_execution_run_id,omitempty"` +} + +func (x *WorkflowExecutionTimedOutEventAttributes) Reset() { + *x = WorkflowExecutionTimedOutEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionTimedOutEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionTimedOutEventAttributes) ProtoMessage() {} + +func (x *WorkflowExecutionTimedOutEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionTimedOutEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionTimedOutEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *WorkflowExecutionTimedOutEventAttributes) GetRetryState() v12.RetryState { + if x != nil { + return x.RetryState + } + return v12.RetryState(0) +} + +func (x *WorkflowExecutionTimedOutEventAttributes) GetNewExecutionRunId() string { + if x != nil { + return x.NewExecutionRunId + } + return "" +} + +type WorkflowExecutionContinuedAsNewEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The run ID of the new workflow started by this continue-as-new + NewExecutionRunId string `protobuf:"bytes,1,opt,name=new_execution_run_id,json=newExecutionRunId,proto3" json:"new_execution_run_id,omitempty"` + WorkflowType *v1.WorkflowType `protobuf:"bytes,2,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + TaskQueue *v11.TaskQueue `protobuf:"bytes,3,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + Input *v1.Payloads `protobuf:"bytes,4,opt,name=input,proto3" json:"input,omitempty"` + // Timeout of a single workflow run. + WorkflowRunTimeout *durationpb.Duration `protobuf:"bytes,5,opt,name=workflow_run_timeout,json=workflowRunTimeout,proto3" json:"workflow_run_timeout,omitempty"` + // Timeout of a single workflow task. + WorkflowTaskTimeout *durationpb.Duration `protobuf:"bytes,6,opt,name=workflow_task_timeout,json=workflowTaskTimeout,proto3" json:"workflow_task_timeout,omitempty"` + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,7,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + // TODO: How and is this used? + BackoffStartInterval *durationpb.Duration `protobuf:"bytes,8,opt,name=backoff_start_interval,json=backoffStartInterval,proto3" json:"backoff_start_interval,omitempty"` + Initiator v12.ContinueAsNewInitiator `protobuf:"varint,9,opt,name=initiator,proto3,enum=temporal.api.enums.v1.ContinueAsNewInitiator" json:"initiator,omitempty"` + // TODO: David are these right? + // Deprecated. If a workflow's retry policy would cause a new run to start when the current one + // has failed, this field would be populated with that failure. Now (when supported by server + // and sdk) the final event will be `WORKFLOW_EXECUTION_FAILED` with `new_execution_run_id` set. + Failure *v13.Failure `protobuf:"bytes,10,opt,name=failure,proto3" json:"failure,omitempty"` + // TODO: Is this the result of *this* workflow as it continued-as-new? + LastCompletionResult *v1.Payloads `protobuf:"bytes,11,opt,name=last_completion_result,json=lastCompletionResult,proto3" json:"last_completion_result,omitempty"` + Header *v1.Header `protobuf:"bytes,12,opt,name=header,proto3" json:"header,omitempty"` + Memo *v1.Memo `protobuf:"bytes,13,opt,name=memo,proto3" json:"memo,omitempty"` + SearchAttributes *v1.SearchAttributes `protobuf:"bytes,14,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` + // If this is set, the new execution inherits the Build ID of the current execution. Otherwise, + // the assignment rules will be used to independently assign a Build ID to the new execution. + InheritBuildId bool `protobuf:"varint,15,opt,name=inherit_build_id,json=inheritBuildId,proto3" json:"inherit_build_id,omitempty"` +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) Reset() { + *x = WorkflowExecutionContinuedAsNewEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionContinuedAsNewEventAttributes) ProtoMessage() {} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionContinuedAsNewEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionContinuedAsNewEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetNewExecutionRunId() string { + if x != nil { + return x.NewExecutionRunId + } + return "" +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetTaskQueue() *v11.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetInput() *v1.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetWorkflowRunTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowRunTimeout + } + return nil +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetWorkflowTaskTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowTaskTimeout + } + return nil +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetBackoffStartInterval() *durationpb.Duration { + if x != nil { + return x.BackoffStartInterval + } + return nil +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetInitiator() v12.ContinueAsNewInitiator { + if x != nil { + return x.Initiator + } + return v12.ContinueAsNewInitiator(0) +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetFailure() *v13.Failure { + if x != nil { + return x.Failure + } + return nil +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetLastCompletionResult() *v1.Payloads { + if x != nil { + return x.LastCompletionResult + } + return nil +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetMemo() *v1.Memo { + if x != nil { + return x.Memo + } + return nil +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetSearchAttributes() *v1.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +func (x *WorkflowExecutionContinuedAsNewEventAttributes) GetInheritBuildId() bool { + if x != nil { + return x.InheritBuildId + } + return false +} + +type WorkflowTaskScheduledEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The task queue this workflow task was enqueued in, which could be a normal or sticky queue + TaskQueue *v11.TaskQueue `protobuf:"bytes,1,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + // How long the worker has to process this task once receiving it before it times out + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + StartToCloseTimeout *durationpb.Duration `protobuf:"bytes,2,opt,name=start_to_close_timeout,json=startToCloseTimeout,proto3" json:"start_to_close_timeout,omitempty"` + // Starting at 1, how many attempts there have been to complete this task + Attempt int32 `protobuf:"varint,3,opt,name=attempt,proto3" json:"attempt,omitempty"` +} + +func (x *WorkflowTaskScheduledEventAttributes) Reset() { + *x = WorkflowTaskScheduledEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowTaskScheduledEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowTaskScheduledEventAttributes) ProtoMessage() {} + +func (x *WorkflowTaskScheduledEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowTaskScheduledEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowTaskScheduledEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{5} +} + +func (x *WorkflowTaskScheduledEventAttributes) GetTaskQueue() *v11.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *WorkflowTaskScheduledEventAttributes) GetStartToCloseTimeout() *durationpb.Duration { + if x != nil { + return x.StartToCloseTimeout + } + return nil +} + +func (x *WorkflowTaskScheduledEventAttributes) GetAttempt() int32 { + if x != nil { + return x.Attempt + } + return 0 +} + +type WorkflowTaskStartedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the `WORKFLOW_TASK_SCHEDULED` event this task corresponds to + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // Identity of the worker who picked up this task + Identity string `protobuf:"bytes,2,opt,name=identity,proto3" json:"identity,omitempty"` + // TODO: ? Appears unused? + RequestId string `protobuf:"bytes,3,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` + // True if this workflow should continue-as-new soon because its history size (in + // either event count or bytes) is getting large. + SuggestContinueAsNew bool `protobuf:"varint,4,opt,name=suggest_continue_as_new,json=suggestContinueAsNew,proto3" json:"suggest_continue_as_new,omitempty"` + // Total history size in bytes, which the workflow might use to decide when to + // continue-as-new regardless of the suggestion. Note that history event count is + // just the event id of this event, so we don't include it explicitly here. + HistorySizeBytes int64 `protobuf:"varint,5,opt,name=history_size_bytes,json=historySizeBytes,proto3" json:"history_size_bytes,omitempty"` + // Version info of the worker to whom this task was dispatched. + WorkerVersion *v1.WorkerVersionStamp `protobuf:"bytes,6,opt,name=worker_version,json=workerVersion,proto3" json:"worker_version,omitempty"` + // Used by server internally to properly reapply build ID redirects to an execution + // when rebuilding it from events. + BuildIdRedirectCounter int64 `protobuf:"varint,7,opt,name=build_id_redirect_counter,json=buildIdRedirectCounter,proto3" json:"build_id_redirect_counter,omitempty"` +} + +func (x *WorkflowTaskStartedEventAttributes) Reset() { + *x = WorkflowTaskStartedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowTaskStartedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowTaskStartedEventAttributes) ProtoMessage() {} + +func (x *WorkflowTaskStartedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowTaskStartedEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowTaskStartedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{6} +} + +func (x *WorkflowTaskStartedEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *WorkflowTaskStartedEventAttributes) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *WorkflowTaskStartedEventAttributes) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +func (x *WorkflowTaskStartedEventAttributes) GetSuggestContinueAsNew() bool { + if x != nil { + return x.SuggestContinueAsNew + } + return false +} + +func (x *WorkflowTaskStartedEventAttributes) GetHistorySizeBytes() int64 { + if x != nil { + return x.HistorySizeBytes + } + return 0 +} + +func (x *WorkflowTaskStartedEventAttributes) GetWorkerVersion() *v1.WorkerVersionStamp { + if x != nil { + return x.WorkerVersion + } + return nil +} + +func (x *WorkflowTaskStartedEventAttributes) GetBuildIdRedirectCounter() int64 { + if x != nil { + return x.BuildIdRedirectCounter + } + return 0 +} + +type WorkflowTaskCompletedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the `WORKFLOW_TASK_SCHEDULED` event this task corresponds to + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // The id of the `WORKFLOW_TASK_STARTED` event this task corresponds to + StartedEventId int64 `protobuf:"varint,2,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` + // Identity of the worker who completed this task + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` + // Binary ID of the worker who completed this task + BinaryChecksum string `protobuf:"bytes,4,opt,name=binary_checksum,json=binaryChecksum,proto3" json:"binary_checksum,omitempty"` + // Version info of the worker who processed this workflow task. If present, the `build_id` field + // within is also used as `binary_checksum`, which may be omitted in that case (it may also be + // populated to preserve compatibility). + // Deprecated. Use the info inside the corresponding WorkflowTaskStartedEvent + WorkerVersion *v1.WorkerVersionStamp `protobuf:"bytes,5,opt,name=worker_version,json=workerVersion,proto3" json:"worker_version,omitempty"` + // Data the SDK wishes to record for itself, but server need not interpret, and does not + // directly impact workflow state. + SdkMetadata *v15.WorkflowTaskCompletedMetadata `protobuf:"bytes,6,opt,name=sdk_metadata,json=sdkMetadata,proto3" json:"sdk_metadata,omitempty"` + // Local usage data sent during workflow task completion and recorded here for posterity + MeteringMetadata *v1.MeteringMetadata `protobuf:"bytes,13,opt,name=metering_metadata,json=meteringMetadata,proto3" json:"metering_metadata,omitempty"` +} + +func (x *WorkflowTaskCompletedEventAttributes) Reset() { + *x = WorkflowTaskCompletedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowTaskCompletedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowTaskCompletedEventAttributes) ProtoMessage() {} + +func (x *WorkflowTaskCompletedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowTaskCompletedEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowTaskCompletedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{7} +} + +func (x *WorkflowTaskCompletedEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *WorkflowTaskCompletedEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +func (x *WorkflowTaskCompletedEventAttributes) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *WorkflowTaskCompletedEventAttributes) GetBinaryChecksum() string { + if x != nil { + return x.BinaryChecksum + } + return "" +} + +func (x *WorkflowTaskCompletedEventAttributes) GetWorkerVersion() *v1.WorkerVersionStamp { + if x != nil { + return x.WorkerVersion + } + return nil +} + +func (x *WorkflowTaskCompletedEventAttributes) GetSdkMetadata() *v15.WorkflowTaskCompletedMetadata { + if x != nil { + return x.SdkMetadata + } + return nil +} + +func (x *WorkflowTaskCompletedEventAttributes) GetMeteringMetadata() *v1.MeteringMetadata { + if x != nil { + return x.MeteringMetadata + } + return nil +} + +type WorkflowTaskTimedOutEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the `WORKFLOW_TASK_SCHEDULED` event this task corresponds to + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // The id of the `WORKFLOW_TASK_STARTED` event this task corresponds to + StartedEventId int64 `protobuf:"varint,2,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` + TimeoutType v12.TimeoutType `protobuf:"varint,3,opt,name=timeout_type,json=timeoutType,proto3,enum=temporal.api.enums.v1.TimeoutType" json:"timeout_type,omitempty"` +} + +func (x *WorkflowTaskTimedOutEventAttributes) Reset() { + *x = WorkflowTaskTimedOutEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowTaskTimedOutEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowTaskTimedOutEventAttributes) ProtoMessage() {} + +func (x *WorkflowTaskTimedOutEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowTaskTimedOutEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowTaskTimedOutEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{8} +} + +func (x *WorkflowTaskTimedOutEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *WorkflowTaskTimedOutEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +func (x *WorkflowTaskTimedOutEventAttributes) GetTimeoutType() v12.TimeoutType { + if x != nil { + return x.TimeoutType + } + return v12.TimeoutType(0) +} + +type WorkflowTaskFailedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the `WORKFLOW_TASK_SCHEDULED` event this task corresponds to + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // The id of the `WORKFLOW_TASK_STARTED` event this task corresponds to + StartedEventId int64 `protobuf:"varint,2,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` + Cause v12.WorkflowTaskFailedCause `protobuf:"varint,3,opt,name=cause,proto3,enum=temporal.api.enums.v1.WorkflowTaskFailedCause" json:"cause,omitempty"` + // The failure details + Failure *v13.Failure `protobuf:"bytes,4,opt,name=failure,proto3" json:"failure,omitempty"` + // If a worker explicitly failed this task, it's identity. TODO: What is this set to if server fails the task? + Identity string `protobuf:"bytes,5,opt,name=identity,proto3" json:"identity,omitempty"` + // The original run id of the workflow. For reset workflow. + BaseRunId string `protobuf:"bytes,6,opt,name=base_run_id,json=baseRunId,proto3" json:"base_run_id,omitempty"` + // If the workflow is being reset, the new run id. + NewRunId string `protobuf:"bytes,7,opt,name=new_run_id,json=newRunId,proto3" json:"new_run_id,omitempty"` + // TODO: ? + ForkEventVersion int64 `protobuf:"varint,8,opt,name=fork_event_version,json=forkEventVersion,proto3" json:"fork_event_version,omitempty"` + // DEPRECATED since 1.21 - use `worker_version` instead. + // If a worker explicitly failed this task, its binary id + BinaryChecksum string `protobuf:"bytes,9,opt,name=binary_checksum,json=binaryChecksum,proto3" json:"binary_checksum,omitempty"` + // Version info of the worker who processed this workflow task. If present, the `build_id` field + // within is also used as `binary_checksum`, which may be omitted in that case (it may also be + // populated to preserve compatibility). + // Deprecated. Use the info inside the corresponding WorkflowTaskStartedEvent + WorkerVersion *v1.WorkerVersionStamp `protobuf:"bytes,10,opt,name=worker_version,json=workerVersion,proto3" json:"worker_version,omitempty"` +} + +func (x *WorkflowTaskFailedEventAttributes) Reset() { + *x = WorkflowTaskFailedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowTaskFailedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowTaskFailedEventAttributes) ProtoMessage() {} + +func (x *WorkflowTaskFailedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowTaskFailedEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowTaskFailedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{9} +} + +func (x *WorkflowTaskFailedEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *WorkflowTaskFailedEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +func (x *WorkflowTaskFailedEventAttributes) GetCause() v12.WorkflowTaskFailedCause { + if x != nil { + return x.Cause + } + return v12.WorkflowTaskFailedCause(0) +} + +func (x *WorkflowTaskFailedEventAttributes) GetFailure() *v13.Failure { + if x != nil { + return x.Failure + } + return nil +} + +func (x *WorkflowTaskFailedEventAttributes) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *WorkflowTaskFailedEventAttributes) GetBaseRunId() string { + if x != nil { + return x.BaseRunId + } + return "" +} + +func (x *WorkflowTaskFailedEventAttributes) GetNewRunId() string { + if x != nil { + return x.NewRunId + } + return "" +} + +func (x *WorkflowTaskFailedEventAttributes) GetForkEventVersion() int64 { + if x != nil { + return x.ForkEventVersion + } + return 0 +} + +func (x *WorkflowTaskFailedEventAttributes) GetBinaryChecksum() string { + if x != nil { + return x.BinaryChecksum + } + return "" +} + +func (x *WorkflowTaskFailedEventAttributes) GetWorkerVersion() *v1.WorkerVersionStamp { + if x != nil { + return x.WorkerVersion + } + return nil +} + +type ActivityTaskScheduledEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The worker/user assigned identifier for the activity + ActivityId string `protobuf:"bytes,1,opt,name=activity_id,json=activityId,proto3" json:"activity_id,omitempty"` + ActivityType *v1.ActivityType `protobuf:"bytes,2,opt,name=activity_type,json=activityType,proto3" json:"activity_type,omitempty"` + TaskQueue *v11.TaskQueue `protobuf:"bytes,4,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + Header *v1.Header `protobuf:"bytes,5,opt,name=header,proto3" json:"header,omitempty"` + Input *v1.Payloads `protobuf:"bytes,6,opt,name=input,proto3" json:"input,omitempty"` + // Indicates how long the caller is willing to wait for an activity completion. Limits how long + // retries will be attempted. Either this or `start_to_close_timeout` must be specified. + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + ScheduleToCloseTimeout *durationpb.Duration `protobuf:"bytes,7,opt,name=schedule_to_close_timeout,json=scheduleToCloseTimeout,proto3" json:"schedule_to_close_timeout,omitempty"` + // Limits time an activity task can stay in a task queue before a worker picks it up. This + // timeout is always non retryable, as all a retry would achieve is to put it back into the same + // queue. Defaults to `schedule_to_close_timeout` or workflow execution timeout if not + // specified. + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + ScheduleToStartTimeout *durationpb.Duration `protobuf:"bytes,8,opt,name=schedule_to_start_timeout,json=scheduleToStartTimeout,proto3" json:"schedule_to_start_timeout,omitempty"` + // Maximum time an activity is allowed to execute after being picked up by a worker. This + // timeout is always retryable. Either this or `schedule_to_close_timeout` must be + // specified. + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + StartToCloseTimeout *durationpb.Duration `protobuf:"bytes,9,opt,name=start_to_close_timeout,json=startToCloseTimeout,proto3" json:"start_to_close_timeout,omitempty"` + // Maximum permitted time between successful worker heartbeats. + HeartbeatTimeout *durationpb.Duration `protobuf:"bytes,10,opt,name=heartbeat_timeout,json=heartbeatTimeout,proto3" json:"heartbeat_timeout,omitempty"` + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,11,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + // Activities are assigned a default retry policy controlled by the service's dynamic + // configuration. Retries will happen up to `schedule_to_close_timeout`. To disable retries set + // retry_policy.maximum_attempts to 1. + RetryPolicy *v1.RetryPolicy `protobuf:"bytes,12,opt,name=retry_policy,json=retryPolicy,proto3" json:"retry_policy,omitempty"` + // If this is set, the activity would be assigned to the Build ID of the workflow. Otherwise, + // Assignment rules of the activity's Task Queue will be used to determine the Build ID. + UseWorkflowBuildId bool `protobuf:"varint,13,opt,name=use_workflow_build_id,json=useWorkflowBuildId,proto3" json:"use_workflow_build_id,omitempty"` +} + +func (x *ActivityTaskScheduledEventAttributes) Reset() { + *x = ActivityTaskScheduledEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActivityTaskScheduledEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActivityTaskScheduledEventAttributes) ProtoMessage() {} + +func (x *ActivityTaskScheduledEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActivityTaskScheduledEventAttributes.ProtoReflect.Descriptor instead. +func (*ActivityTaskScheduledEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{10} +} + +func (x *ActivityTaskScheduledEventAttributes) GetActivityId() string { + if x != nil { + return x.ActivityId + } + return "" +} + +func (x *ActivityTaskScheduledEventAttributes) GetActivityType() *v1.ActivityType { + if x != nil { + return x.ActivityType + } + return nil +} + +func (x *ActivityTaskScheduledEventAttributes) GetTaskQueue() *v11.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *ActivityTaskScheduledEventAttributes) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *ActivityTaskScheduledEventAttributes) GetInput() *v1.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *ActivityTaskScheduledEventAttributes) GetScheduleToCloseTimeout() *durationpb.Duration { + if x != nil { + return x.ScheduleToCloseTimeout + } + return nil +} + +func (x *ActivityTaskScheduledEventAttributes) GetScheduleToStartTimeout() *durationpb.Duration { + if x != nil { + return x.ScheduleToStartTimeout + } + return nil +} + +func (x *ActivityTaskScheduledEventAttributes) GetStartToCloseTimeout() *durationpb.Duration { + if x != nil { + return x.StartToCloseTimeout + } + return nil +} + +func (x *ActivityTaskScheduledEventAttributes) GetHeartbeatTimeout() *durationpb.Duration { + if x != nil { + return x.HeartbeatTimeout + } + return nil +} + +func (x *ActivityTaskScheduledEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *ActivityTaskScheduledEventAttributes) GetRetryPolicy() *v1.RetryPolicy { + if x != nil { + return x.RetryPolicy + } + return nil +} + +func (x *ActivityTaskScheduledEventAttributes) GetUseWorkflowBuildId() bool { + if x != nil { + return x.UseWorkflowBuildId + } + return false +} + +type ActivityTaskStartedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the `ACTIVITY_TASK_SCHEDULED` event this task corresponds to + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // id of the worker that picked up this task + Identity string `protobuf:"bytes,2,opt,name=identity,proto3" json:"identity,omitempty"` + // TODO ?? + RequestId string `protobuf:"bytes,3,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` + // Starting at 1, the number of times this task has been attempted + Attempt int32 `protobuf:"varint,4,opt,name=attempt,proto3" json:"attempt,omitempty"` + // Will be set to the most recent failure details, if this task has previously failed and then + // been retried. + LastFailure *v13.Failure `protobuf:"bytes,5,opt,name=last_failure,json=lastFailure,proto3" json:"last_failure,omitempty"` + // Version info of the worker to whom this task was dispatched. + WorkerVersion *v1.WorkerVersionStamp `protobuf:"bytes,6,opt,name=worker_version,json=workerVersion,proto3" json:"worker_version,omitempty"` + // Used by server internally to properly reapply build ID redirects to an execution + // when rebuilding it from events. + BuildIdRedirectCounter int64 `protobuf:"varint,7,opt,name=build_id_redirect_counter,json=buildIdRedirectCounter,proto3" json:"build_id_redirect_counter,omitempty"` +} + +func (x *ActivityTaskStartedEventAttributes) Reset() { + *x = ActivityTaskStartedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActivityTaskStartedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActivityTaskStartedEventAttributes) ProtoMessage() {} + +func (x *ActivityTaskStartedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActivityTaskStartedEventAttributes.ProtoReflect.Descriptor instead. +func (*ActivityTaskStartedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{11} +} + +func (x *ActivityTaskStartedEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *ActivityTaskStartedEventAttributes) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *ActivityTaskStartedEventAttributes) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +func (x *ActivityTaskStartedEventAttributes) GetAttempt() int32 { + if x != nil { + return x.Attempt + } + return 0 +} + +func (x *ActivityTaskStartedEventAttributes) GetLastFailure() *v13.Failure { + if x != nil { + return x.LastFailure + } + return nil +} + +func (x *ActivityTaskStartedEventAttributes) GetWorkerVersion() *v1.WorkerVersionStamp { + if x != nil { + return x.WorkerVersion + } + return nil +} + +func (x *ActivityTaskStartedEventAttributes) GetBuildIdRedirectCounter() int64 { + if x != nil { + return x.BuildIdRedirectCounter + } + return 0 +} + +type ActivityTaskCompletedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Serialized results of the activity. IE: The return value of the activity function + Result *v1.Payloads `protobuf:"bytes,1,opt,name=result,proto3" json:"result,omitempty"` + // The id of the `ACTIVITY_TASK_SCHEDULED` event this completion corresponds to + ScheduledEventId int64 `protobuf:"varint,2,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // The id of the `ACTIVITY_TASK_STARTED` event this completion corresponds to + StartedEventId int64 `protobuf:"varint,3,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` + // id of the worker that completed this task + Identity string `protobuf:"bytes,4,opt,name=identity,proto3" json:"identity,omitempty"` + // Version info of the worker who processed this workflow task. + // Deprecated. Use the info inside the corresponding ActivityTaskStartedEvent + WorkerVersion *v1.WorkerVersionStamp `protobuf:"bytes,5,opt,name=worker_version,json=workerVersion,proto3" json:"worker_version,omitempty"` +} + +func (x *ActivityTaskCompletedEventAttributes) Reset() { + *x = ActivityTaskCompletedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActivityTaskCompletedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActivityTaskCompletedEventAttributes) ProtoMessage() {} + +func (x *ActivityTaskCompletedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActivityTaskCompletedEventAttributes.ProtoReflect.Descriptor instead. +func (*ActivityTaskCompletedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{12} +} + +func (x *ActivityTaskCompletedEventAttributes) GetResult() *v1.Payloads { + if x != nil { + return x.Result + } + return nil +} + +func (x *ActivityTaskCompletedEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *ActivityTaskCompletedEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +func (x *ActivityTaskCompletedEventAttributes) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *ActivityTaskCompletedEventAttributes) GetWorkerVersion() *v1.WorkerVersionStamp { + if x != nil { + return x.WorkerVersion + } + return nil +} + +type ActivityTaskFailedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Failure details + Failure *v13.Failure `protobuf:"bytes,1,opt,name=failure,proto3" json:"failure,omitempty"` + // The id of the `ACTIVITY_TASK_SCHEDULED` event this failure corresponds to + ScheduledEventId int64 `protobuf:"varint,2,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // The id of the `ACTIVITY_TASK_STARTED` event this failure corresponds to + StartedEventId int64 `protobuf:"varint,3,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` + // id of the worker that failed this task + Identity string `protobuf:"bytes,4,opt,name=identity,proto3" json:"identity,omitempty"` + RetryState v12.RetryState `protobuf:"varint,5,opt,name=retry_state,json=retryState,proto3,enum=temporal.api.enums.v1.RetryState" json:"retry_state,omitempty"` + // Version info of the worker who processed this workflow task. + // Deprecated. Use the info inside the corresponding ActivityTaskStartedEvent + WorkerVersion *v1.WorkerVersionStamp `protobuf:"bytes,6,opt,name=worker_version,json=workerVersion,proto3" json:"worker_version,omitempty"` +} + +func (x *ActivityTaskFailedEventAttributes) Reset() { + *x = ActivityTaskFailedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActivityTaskFailedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActivityTaskFailedEventAttributes) ProtoMessage() {} + +func (x *ActivityTaskFailedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActivityTaskFailedEventAttributes.ProtoReflect.Descriptor instead. +func (*ActivityTaskFailedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{13} +} + +func (x *ActivityTaskFailedEventAttributes) GetFailure() *v13.Failure { + if x != nil { + return x.Failure + } + return nil +} + +func (x *ActivityTaskFailedEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *ActivityTaskFailedEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +func (x *ActivityTaskFailedEventAttributes) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *ActivityTaskFailedEventAttributes) GetRetryState() v12.RetryState { + if x != nil { + return x.RetryState + } + return v12.RetryState(0) +} + +func (x *ActivityTaskFailedEventAttributes) GetWorkerVersion() *v1.WorkerVersionStamp { + if x != nil { + return x.WorkerVersion + } + return nil +} + +type ActivityTaskTimedOutEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // If this activity had failed, was retried, and then timed out, that failure is stored as the + // `cause` in here. + Failure *v13.Failure `protobuf:"bytes,1,opt,name=failure,proto3" json:"failure,omitempty"` + // The id of the `ACTIVITY_TASK_SCHEDULED` event this timeout corresponds to + ScheduledEventId int64 `protobuf:"varint,2,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // The id of the `ACTIVITY_TASK_STARTED` event this timeout corresponds to + StartedEventId int64 `protobuf:"varint,3,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` + RetryState v12.RetryState `protobuf:"varint,4,opt,name=retry_state,json=retryState,proto3,enum=temporal.api.enums.v1.RetryState" json:"retry_state,omitempty"` +} + +func (x *ActivityTaskTimedOutEventAttributes) Reset() { + *x = ActivityTaskTimedOutEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActivityTaskTimedOutEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActivityTaskTimedOutEventAttributes) ProtoMessage() {} + +func (x *ActivityTaskTimedOutEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActivityTaskTimedOutEventAttributes.ProtoReflect.Descriptor instead. +func (*ActivityTaskTimedOutEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{14} +} + +func (x *ActivityTaskTimedOutEventAttributes) GetFailure() *v13.Failure { + if x != nil { + return x.Failure + } + return nil +} + +func (x *ActivityTaskTimedOutEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *ActivityTaskTimedOutEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +func (x *ActivityTaskTimedOutEventAttributes) GetRetryState() v12.RetryState { + if x != nil { + return x.RetryState + } + return v12.RetryState(0) +} + +type ActivityTaskCancelRequestedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the `ACTIVITY_TASK_SCHEDULED` event this cancel request corresponds to + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,2,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` +} + +func (x *ActivityTaskCancelRequestedEventAttributes) Reset() { + *x = ActivityTaskCancelRequestedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActivityTaskCancelRequestedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActivityTaskCancelRequestedEventAttributes) ProtoMessage() {} + +func (x *ActivityTaskCancelRequestedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[15] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActivityTaskCancelRequestedEventAttributes.ProtoReflect.Descriptor instead. +func (*ActivityTaskCancelRequestedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{15} +} + +func (x *ActivityTaskCancelRequestedEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *ActivityTaskCancelRequestedEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +type ActivityTaskCanceledEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Additional information that the activity reported upon confirming cancellation + Details *v1.Payloads `protobuf:"bytes,1,opt,name=details,proto3" json:"details,omitempty"` + // id of the most recent `ACTIVITY_TASK_CANCEL_REQUESTED` event which refers to the same + // activity + LatestCancelRequestedEventId int64 `protobuf:"varint,2,opt,name=latest_cancel_requested_event_id,json=latestCancelRequestedEventId,proto3" json:"latest_cancel_requested_event_id,omitempty"` + // The id of the `ACTIVITY_TASK_SCHEDULED` event this cancel confirmation corresponds to + ScheduledEventId int64 `protobuf:"varint,3,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // The id of the `ACTIVITY_TASK_STARTED` event this cancel confirmation corresponds to + StartedEventId int64 `protobuf:"varint,4,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` + // id of the worker who canceled this activity + Identity string `protobuf:"bytes,5,opt,name=identity,proto3" json:"identity,omitempty"` + // Version info of the worker who processed this workflow task. + // Deprecated. Use the info inside the corresponding ActivityTaskStartedEvent + WorkerVersion *v1.WorkerVersionStamp `protobuf:"bytes,6,opt,name=worker_version,json=workerVersion,proto3" json:"worker_version,omitempty"` +} + +func (x *ActivityTaskCanceledEventAttributes) Reset() { + *x = ActivityTaskCanceledEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActivityTaskCanceledEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActivityTaskCanceledEventAttributes) ProtoMessage() {} + +func (x *ActivityTaskCanceledEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[16] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActivityTaskCanceledEventAttributes.ProtoReflect.Descriptor instead. +func (*ActivityTaskCanceledEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{16} +} + +func (x *ActivityTaskCanceledEventAttributes) GetDetails() *v1.Payloads { + if x != nil { + return x.Details + } + return nil +} + +func (x *ActivityTaskCanceledEventAttributes) GetLatestCancelRequestedEventId() int64 { + if x != nil { + return x.LatestCancelRequestedEventId + } + return 0 +} + +func (x *ActivityTaskCanceledEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *ActivityTaskCanceledEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +func (x *ActivityTaskCanceledEventAttributes) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *ActivityTaskCanceledEventAttributes) GetWorkerVersion() *v1.WorkerVersionStamp { + if x != nil { + return x.WorkerVersion + } + return nil +} + +type TimerStartedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The worker/user assigned id for this timer + TimerId string `protobuf:"bytes,1,opt,name=timer_id,json=timerId,proto3" json:"timer_id,omitempty"` + // How long until this timer fires + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + StartToFireTimeout *durationpb.Duration `protobuf:"bytes,2,opt,name=start_to_fire_timeout,json=startToFireTimeout,proto3" json:"start_to_fire_timeout,omitempty"` + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,3,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` +} + +func (x *TimerStartedEventAttributes) Reset() { + *x = TimerStartedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TimerStartedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimerStartedEventAttributes) ProtoMessage() {} + +func (x *TimerStartedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[17] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TimerStartedEventAttributes.ProtoReflect.Descriptor instead. +func (*TimerStartedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{17} +} + +func (x *TimerStartedEventAttributes) GetTimerId() string { + if x != nil { + return x.TimerId + } + return "" +} + +func (x *TimerStartedEventAttributes) GetStartToFireTimeout() *durationpb.Duration { + if x != nil { + return x.StartToFireTimeout + } + return nil +} + +func (x *TimerStartedEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +type TimerFiredEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Will match the `timer_id` from `TIMER_STARTED` event for this timer + TimerId string `protobuf:"bytes,1,opt,name=timer_id,json=timerId,proto3" json:"timer_id,omitempty"` + // The id of the `TIMER_STARTED` event itself + StartedEventId int64 `protobuf:"varint,2,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` +} + +func (x *TimerFiredEventAttributes) Reset() { + *x = TimerFiredEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TimerFiredEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimerFiredEventAttributes) ProtoMessage() {} + +func (x *TimerFiredEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[18] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TimerFiredEventAttributes.ProtoReflect.Descriptor instead. +func (*TimerFiredEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{18} +} + +func (x *TimerFiredEventAttributes) GetTimerId() string { + if x != nil { + return x.TimerId + } + return "" +} + +func (x *TimerFiredEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +type TimerCanceledEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Will match the `timer_id` from `TIMER_STARTED` event for this timer + TimerId string `protobuf:"bytes,1,opt,name=timer_id,json=timerId,proto3" json:"timer_id,omitempty"` + // The id of the `TIMER_STARTED` event itself + StartedEventId int64 `protobuf:"varint,2,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,3,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + // The id of the worker who requested this cancel + Identity string `protobuf:"bytes,4,opt,name=identity,proto3" json:"identity,omitempty"` +} + +func (x *TimerCanceledEventAttributes) Reset() { + *x = TimerCanceledEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TimerCanceledEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimerCanceledEventAttributes) ProtoMessage() {} + +func (x *TimerCanceledEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[19] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TimerCanceledEventAttributes.ProtoReflect.Descriptor instead. +func (*TimerCanceledEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{19} +} + +func (x *TimerCanceledEventAttributes) GetTimerId() string { + if x != nil { + return x.TimerId + } + return "" +} + +func (x *TimerCanceledEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +func (x *TimerCanceledEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *TimerCanceledEventAttributes) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +type WorkflowExecutionCancelRequestedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // User provided reason for requesting cancellation + // TODO: shall we create a new field with name "reason" and deprecate this one? + Cause string `protobuf:"bytes,1,opt,name=cause,proto3" json:"cause,omitempty"` + // TODO: Is this the ID of the event in the workflow which initiated this cancel, if there was one? + ExternalInitiatedEventId int64 `protobuf:"varint,2,opt,name=external_initiated_event_id,json=externalInitiatedEventId,proto3" json:"external_initiated_event_id,omitempty"` + ExternalWorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,3,opt,name=external_workflow_execution,json=externalWorkflowExecution,proto3" json:"external_workflow_execution,omitempty"` + // id of the worker or client who requested this cancel + Identity string `protobuf:"bytes,4,opt,name=identity,proto3" json:"identity,omitempty"` +} + +func (x *WorkflowExecutionCancelRequestedEventAttributes) Reset() { + *x = WorkflowExecutionCancelRequestedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionCancelRequestedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionCancelRequestedEventAttributes) ProtoMessage() {} + +func (x *WorkflowExecutionCancelRequestedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[20] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionCancelRequestedEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionCancelRequestedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{20} +} + +func (x *WorkflowExecutionCancelRequestedEventAttributes) GetCause() string { + if x != nil { + return x.Cause + } + return "" +} + +func (x *WorkflowExecutionCancelRequestedEventAttributes) GetExternalInitiatedEventId() int64 { + if x != nil { + return x.ExternalInitiatedEventId + } + return 0 +} + +func (x *WorkflowExecutionCancelRequestedEventAttributes) GetExternalWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.ExternalWorkflowExecution + } + return nil +} + +func (x *WorkflowExecutionCancelRequestedEventAttributes) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +type WorkflowExecutionCanceledEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,1,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + Details *v1.Payloads `protobuf:"bytes,2,opt,name=details,proto3" json:"details,omitempty"` +} + +func (x *WorkflowExecutionCanceledEventAttributes) Reset() { + *x = WorkflowExecutionCanceledEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionCanceledEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionCanceledEventAttributes) ProtoMessage() {} + +func (x *WorkflowExecutionCanceledEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[21] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionCanceledEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionCanceledEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{21} +} + +func (x *WorkflowExecutionCanceledEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *WorkflowExecutionCanceledEventAttributes) GetDetails() *v1.Payloads { + if x != nil { + return x.Details + } + return nil +} + +type MarkerRecordedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Workers use this to identify the "types" of various markers. Ex: Local activity, side effect. + MarkerName string `protobuf:"bytes,1,opt,name=marker_name,json=markerName,proto3" json:"marker_name,omitempty"` + // Serialized information recorded in the marker + Details map[string]*v1.Payloads `protobuf:"bytes,2,rep,name=details,proto3" json:"details,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,3,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + Header *v1.Header `protobuf:"bytes,4,opt,name=header,proto3" json:"header,omitempty"` + // Some uses of markers, like a local activity, could "fail". If they did that is recorded here. + Failure *v13.Failure `protobuf:"bytes,5,opt,name=failure,proto3" json:"failure,omitempty"` +} + +func (x *MarkerRecordedEventAttributes) Reset() { + *x = MarkerRecordedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MarkerRecordedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MarkerRecordedEventAttributes) ProtoMessage() {} + +func (x *MarkerRecordedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[22] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MarkerRecordedEventAttributes.ProtoReflect.Descriptor instead. +func (*MarkerRecordedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{22} +} + +func (x *MarkerRecordedEventAttributes) GetMarkerName() string { + if x != nil { + return x.MarkerName + } + return "" +} + +func (x *MarkerRecordedEventAttributes) GetDetails() map[string]*v1.Payloads { + if x != nil { + return x.Details + } + return nil +} + +func (x *MarkerRecordedEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *MarkerRecordedEventAttributes) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *MarkerRecordedEventAttributes) GetFailure() *v13.Failure { + if x != nil { + return x.Failure + } + return nil +} + +type WorkflowExecutionSignaledEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The name/type of the signal to fire + SignalName string `protobuf:"bytes,1,opt,name=signal_name,json=signalName,proto3" json:"signal_name,omitempty"` + // Will be deserialized and provided as argument(s) to the signal handler + Input *v1.Payloads `protobuf:"bytes,2,opt,name=input,proto3" json:"input,omitempty"` + // id of the worker/client who sent this signal + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` + // Headers that were passed by the sender of the signal and copied by temporal + // server into the workflow task. + Header *v1.Header `protobuf:"bytes,4,opt,name=header,proto3" json:"header,omitempty"` + // Indicates the signal did not generate a new workflow task when received. + SkipGenerateWorkflowTask bool `protobuf:"varint,5,opt,name=skip_generate_workflow_task,json=skipGenerateWorkflowTask,proto3" json:"skip_generate_workflow_task,omitempty"` + // When signal origin is a workflow execution, this field is set. + ExternalWorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,6,opt,name=external_workflow_execution,json=externalWorkflowExecution,proto3" json:"external_workflow_execution,omitempty"` +} + +func (x *WorkflowExecutionSignaledEventAttributes) Reset() { + *x = WorkflowExecutionSignaledEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionSignaledEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionSignaledEventAttributes) ProtoMessage() {} + +func (x *WorkflowExecutionSignaledEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[23] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionSignaledEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionSignaledEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{23} +} + +func (x *WorkflowExecutionSignaledEventAttributes) GetSignalName() string { + if x != nil { + return x.SignalName + } + return "" +} + +func (x *WorkflowExecutionSignaledEventAttributes) GetInput() *v1.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *WorkflowExecutionSignaledEventAttributes) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *WorkflowExecutionSignaledEventAttributes) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *WorkflowExecutionSignaledEventAttributes) GetSkipGenerateWorkflowTask() bool { + if x != nil { + return x.SkipGenerateWorkflowTask + } + return false +} + +func (x *WorkflowExecutionSignaledEventAttributes) GetExternalWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.ExternalWorkflowExecution + } + return nil +} + +type WorkflowExecutionTerminatedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // User/client provided reason for termination + Reason string `protobuf:"bytes,1,opt,name=reason,proto3" json:"reason,omitempty"` + Details *v1.Payloads `protobuf:"bytes,2,opt,name=details,proto3" json:"details,omitempty"` + // id of the client who requested termination + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` +} + +func (x *WorkflowExecutionTerminatedEventAttributes) Reset() { + *x = WorkflowExecutionTerminatedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionTerminatedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionTerminatedEventAttributes) ProtoMessage() {} + +func (x *WorkflowExecutionTerminatedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[24] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionTerminatedEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionTerminatedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{24} +} + +func (x *WorkflowExecutionTerminatedEventAttributes) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *WorkflowExecutionTerminatedEventAttributes) GetDetails() *v1.Payloads { + if x != nil { + return x.Details + } + return nil +} + +func (x *WorkflowExecutionTerminatedEventAttributes) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +type RequestCancelExternalWorkflowExecutionInitiatedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,1,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + // The namespace the workflow to be cancelled lives in. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,7,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,3,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + // Deprecated + Control string `protobuf:"bytes,4,opt,name=control,proto3" json:"control,omitempty"` + // Workers are expected to set this to true if the workflow they are requesting to cancel is + // a child of the workflow which issued the request + ChildWorkflowOnly bool `protobuf:"varint,5,opt,name=child_workflow_only,json=childWorkflowOnly,proto3" json:"child_workflow_only,omitempty"` + // Reason for requesting the cancellation + Reason string `protobuf:"bytes,6,opt,name=reason,proto3" json:"reason,omitempty"` +} + +func (x *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) Reset() { + *x = RequestCancelExternalWorkflowExecutionInitiatedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) ProtoMessage() {} + +func (x *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[25] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RequestCancelExternalWorkflowExecutionInitiatedEventAttributes.ProtoReflect.Descriptor instead. +func (*RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{25} +} + +func (x *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) GetWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) GetControl() string { + if x != nil { + return x.Control + } + return "" +} + +func (x *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) GetChildWorkflowOnly() bool { + if x != nil { + return x.ChildWorkflowOnly + } + return false +} + +func (x *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +type RequestCancelExternalWorkflowExecutionFailedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Cause v12.CancelExternalWorkflowExecutionFailedCause `protobuf:"varint,1,opt,name=cause,proto3,enum=temporal.api.enums.v1.CancelExternalWorkflowExecutionFailedCause" json:"cause,omitempty"` + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,2,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + // Namespace of the workflow which failed to cancel. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,3,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,7,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,4,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + // id of the `REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED` event this failure + // corresponds to + InitiatedEventId int64 `protobuf:"varint,5,opt,name=initiated_event_id,json=initiatedEventId,proto3" json:"initiated_event_id,omitempty"` + // Deprecated + Control string `protobuf:"bytes,6,opt,name=control,proto3" json:"control,omitempty"` +} + +func (x *RequestCancelExternalWorkflowExecutionFailedEventAttributes) Reset() { + *x = RequestCancelExternalWorkflowExecutionFailedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RequestCancelExternalWorkflowExecutionFailedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RequestCancelExternalWorkflowExecutionFailedEventAttributes) ProtoMessage() {} + +func (x *RequestCancelExternalWorkflowExecutionFailedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[26] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RequestCancelExternalWorkflowExecutionFailedEventAttributes.ProtoReflect.Descriptor instead. +func (*RequestCancelExternalWorkflowExecutionFailedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{26} +} + +func (x *RequestCancelExternalWorkflowExecutionFailedEventAttributes) GetCause() v12.CancelExternalWorkflowExecutionFailedCause { + if x != nil { + return x.Cause + } + return v12.CancelExternalWorkflowExecutionFailedCause(0) +} + +func (x *RequestCancelExternalWorkflowExecutionFailedEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *RequestCancelExternalWorkflowExecutionFailedEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RequestCancelExternalWorkflowExecutionFailedEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *RequestCancelExternalWorkflowExecutionFailedEventAttributes) GetWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *RequestCancelExternalWorkflowExecutionFailedEventAttributes) GetInitiatedEventId() int64 { + if x != nil { + return x.InitiatedEventId + } + return 0 +} + +func (x *RequestCancelExternalWorkflowExecutionFailedEventAttributes) GetControl() string { + if x != nil { + return x.Control + } + return "" +} + +type ExternalWorkflowExecutionCancelRequestedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // id of the `REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED` event this event corresponds + // to + InitiatedEventId int64 `protobuf:"varint,1,opt,name=initiated_event_id,json=initiatedEventId,proto3" json:"initiated_event_id,omitempty"` + // Namespace of the to-be-cancelled workflow. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,4,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,3,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` +} + +func (x *ExternalWorkflowExecutionCancelRequestedEventAttributes) Reset() { + *x = ExternalWorkflowExecutionCancelRequestedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ExternalWorkflowExecutionCancelRequestedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExternalWorkflowExecutionCancelRequestedEventAttributes) ProtoMessage() {} + +func (x *ExternalWorkflowExecutionCancelRequestedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[27] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ExternalWorkflowExecutionCancelRequestedEventAttributes.ProtoReflect.Descriptor instead. +func (*ExternalWorkflowExecutionCancelRequestedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{27} +} + +func (x *ExternalWorkflowExecutionCancelRequestedEventAttributes) GetInitiatedEventId() int64 { + if x != nil { + return x.InitiatedEventId + } + return 0 +} + +func (x *ExternalWorkflowExecutionCancelRequestedEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ExternalWorkflowExecutionCancelRequestedEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *ExternalWorkflowExecutionCancelRequestedEventAttributes) GetWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +type SignalExternalWorkflowExecutionInitiatedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,1,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + // Namespace of the to-be-signalled workflow. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,9,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,3,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + // name/type of the signal to fire in the external workflow + SignalName string `protobuf:"bytes,4,opt,name=signal_name,json=signalName,proto3" json:"signal_name,omitempty"` + // Serialized arguments to provide to the signal handler + Input *v1.Payloads `protobuf:"bytes,5,opt,name=input,proto3" json:"input,omitempty"` + // Deprecated + Control string `protobuf:"bytes,6,opt,name=control,proto3" json:"control,omitempty"` + // Workers are expected to set this to true if the workflow they are requesting to cancel is + // a child of the workflow which issued the request + ChildWorkflowOnly bool `protobuf:"varint,7,opt,name=child_workflow_only,json=childWorkflowOnly,proto3" json:"child_workflow_only,omitempty"` + Header *v1.Header `protobuf:"bytes,8,opt,name=header,proto3" json:"header,omitempty"` +} + +func (x *SignalExternalWorkflowExecutionInitiatedEventAttributes) Reset() { + *x = SignalExternalWorkflowExecutionInitiatedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SignalExternalWorkflowExecutionInitiatedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SignalExternalWorkflowExecutionInitiatedEventAttributes) ProtoMessage() {} + +func (x *SignalExternalWorkflowExecutionInitiatedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[28] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SignalExternalWorkflowExecutionInitiatedEventAttributes.ProtoReflect.Descriptor instead. +func (*SignalExternalWorkflowExecutionInitiatedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{28} +} + +func (x *SignalExternalWorkflowExecutionInitiatedEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *SignalExternalWorkflowExecutionInitiatedEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *SignalExternalWorkflowExecutionInitiatedEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *SignalExternalWorkflowExecutionInitiatedEventAttributes) GetWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *SignalExternalWorkflowExecutionInitiatedEventAttributes) GetSignalName() string { + if x != nil { + return x.SignalName + } + return "" +} + +func (x *SignalExternalWorkflowExecutionInitiatedEventAttributes) GetInput() *v1.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *SignalExternalWorkflowExecutionInitiatedEventAttributes) GetControl() string { + if x != nil { + return x.Control + } + return "" +} + +func (x *SignalExternalWorkflowExecutionInitiatedEventAttributes) GetChildWorkflowOnly() bool { + if x != nil { + return x.ChildWorkflowOnly + } + return false +} + +func (x *SignalExternalWorkflowExecutionInitiatedEventAttributes) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +type SignalExternalWorkflowExecutionFailedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Cause v12.SignalExternalWorkflowExecutionFailedCause `protobuf:"varint,1,opt,name=cause,proto3,enum=temporal.api.enums.v1.SignalExternalWorkflowExecutionFailedCause" json:"cause,omitempty"` + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,2,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + // Namespace of the workflow which failed the signal. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,3,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,7,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,4,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + InitiatedEventId int64 `protobuf:"varint,5,opt,name=initiated_event_id,json=initiatedEventId,proto3" json:"initiated_event_id,omitempty"` + // Deprecated + Control string `protobuf:"bytes,6,opt,name=control,proto3" json:"control,omitempty"` +} + +func (x *SignalExternalWorkflowExecutionFailedEventAttributes) Reset() { + *x = SignalExternalWorkflowExecutionFailedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SignalExternalWorkflowExecutionFailedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SignalExternalWorkflowExecutionFailedEventAttributes) ProtoMessage() {} + +func (x *SignalExternalWorkflowExecutionFailedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[29] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SignalExternalWorkflowExecutionFailedEventAttributes.ProtoReflect.Descriptor instead. +func (*SignalExternalWorkflowExecutionFailedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{29} +} + +func (x *SignalExternalWorkflowExecutionFailedEventAttributes) GetCause() v12.SignalExternalWorkflowExecutionFailedCause { + if x != nil { + return x.Cause + } + return v12.SignalExternalWorkflowExecutionFailedCause(0) +} + +func (x *SignalExternalWorkflowExecutionFailedEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *SignalExternalWorkflowExecutionFailedEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *SignalExternalWorkflowExecutionFailedEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *SignalExternalWorkflowExecutionFailedEventAttributes) GetWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *SignalExternalWorkflowExecutionFailedEventAttributes) GetInitiatedEventId() int64 { + if x != nil { + return x.InitiatedEventId + } + return 0 +} + +func (x *SignalExternalWorkflowExecutionFailedEventAttributes) GetControl() string { + if x != nil { + return x.Control + } + return "" +} + +type ExternalWorkflowExecutionSignaledEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // id of the `SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED` event this event corresponds to + InitiatedEventId int64 `protobuf:"varint,1,opt,name=initiated_event_id,json=initiatedEventId,proto3" json:"initiated_event_id,omitempty"` + // Namespace of the workflow which was signaled. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,5,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,3,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + // Deprecated + Control string `protobuf:"bytes,4,opt,name=control,proto3" json:"control,omitempty"` +} + +func (x *ExternalWorkflowExecutionSignaledEventAttributes) Reset() { + *x = ExternalWorkflowExecutionSignaledEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ExternalWorkflowExecutionSignaledEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExternalWorkflowExecutionSignaledEventAttributes) ProtoMessage() {} + +func (x *ExternalWorkflowExecutionSignaledEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[30] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ExternalWorkflowExecutionSignaledEventAttributes.ProtoReflect.Descriptor instead. +func (*ExternalWorkflowExecutionSignaledEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{30} +} + +func (x *ExternalWorkflowExecutionSignaledEventAttributes) GetInitiatedEventId() int64 { + if x != nil { + return x.InitiatedEventId + } + return 0 +} + +func (x *ExternalWorkflowExecutionSignaledEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ExternalWorkflowExecutionSignaledEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *ExternalWorkflowExecutionSignaledEventAttributes) GetWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *ExternalWorkflowExecutionSignaledEventAttributes) GetControl() string { + if x != nil { + return x.Control + } + return "" +} + +type UpsertWorkflowSearchAttributesEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,1,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + SearchAttributes *v1.SearchAttributes `protobuf:"bytes,2,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` +} + +func (x *UpsertWorkflowSearchAttributesEventAttributes) Reset() { + *x = UpsertWorkflowSearchAttributesEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpsertWorkflowSearchAttributesEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpsertWorkflowSearchAttributesEventAttributes) ProtoMessage() {} + +func (x *UpsertWorkflowSearchAttributesEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[31] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpsertWorkflowSearchAttributesEventAttributes.ProtoReflect.Descriptor instead. +func (*UpsertWorkflowSearchAttributesEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{31} +} + +func (x *UpsertWorkflowSearchAttributesEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *UpsertWorkflowSearchAttributesEventAttributes) GetSearchAttributes() *v1.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +type WorkflowPropertiesModifiedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,1,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + // If set, update the workflow memo with the provided values. The values will be merged with + // the existing memo. If the user wants to delete values, a default/empty Payload should be + // used as the value for the key being deleted. + UpsertedMemo *v1.Memo `protobuf:"bytes,2,opt,name=upserted_memo,json=upsertedMemo,proto3" json:"upserted_memo,omitempty"` +} + +func (x *WorkflowPropertiesModifiedEventAttributes) Reset() { + *x = WorkflowPropertiesModifiedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowPropertiesModifiedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowPropertiesModifiedEventAttributes) ProtoMessage() {} + +func (x *WorkflowPropertiesModifiedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[32] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowPropertiesModifiedEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowPropertiesModifiedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{32} +} + +func (x *WorkflowPropertiesModifiedEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *WorkflowPropertiesModifiedEventAttributes) GetUpsertedMemo() *v1.Memo { + if x != nil { + return x.UpsertedMemo + } + return nil +} + +type StartChildWorkflowExecutionInitiatedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace of the child workflow. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,18,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowId string `protobuf:"bytes,2,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + WorkflowType *v1.WorkflowType `protobuf:"bytes,3,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + TaskQueue *v11.TaskQueue `protobuf:"bytes,4,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + Input *v1.Payloads `protobuf:"bytes,5,opt,name=input,proto3" json:"input,omitempty"` + // Total workflow execution timeout including retries and continue as new. + WorkflowExecutionTimeout *durationpb.Duration `protobuf:"bytes,6,opt,name=workflow_execution_timeout,json=workflowExecutionTimeout,proto3" json:"workflow_execution_timeout,omitempty"` + // Timeout of a single workflow run. + WorkflowRunTimeout *durationpb.Duration `protobuf:"bytes,7,opt,name=workflow_run_timeout,json=workflowRunTimeout,proto3" json:"workflow_run_timeout,omitempty"` + // Timeout of a single workflow task. + WorkflowTaskTimeout *durationpb.Duration `protobuf:"bytes,8,opt,name=workflow_task_timeout,json=workflowTaskTimeout,proto3" json:"workflow_task_timeout,omitempty"` + // Default: PARENT_CLOSE_POLICY_TERMINATE. + ParentClosePolicy v12.ParentClosePolicy `protobuf:"varint,9,opt,name=parent_close_policy,json=parentClosePolicy,proto3,enum=temporal.api.enums.v1.ParentClosePolicy" json:"parent_close_policy,omitempty"` + // Deprecated + Control string `protobuf:"bytes,10,opt,name=control,proto3" json:"control,omitempty"` + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,11,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + // Default: WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE. + WorkflowIdReusePolicy v12.WorkflowIdReusePolicy `protobuf:"varint,12,opt,name=workflow_id_reuse_policy,json=workflowIdReusePolicy,proto3,enum=temporal.api.enums.v1.WorkflowIdReusePolicy" json:"workflow_id_reuse_policy,omitempty"` + RetryPolicy *v1.RetryPolicy `protobuf:"bytes,13,opt,name=retry_policy,json=retryPolicy,proto3" json:"retry_policy,omitempty"` + // If this child runs on a cron schedule, it will appear here + CronSchedule string `protobuf:"bytes,14,opt,name=cron_schedule,json=cronSchedule,proto3" json:"cron_schedule,omitempty"` + Header *v1.Header `protobuf:"bytes,15,opt,name=header,proto3" json:"header,omitempty"` + Memo *v1.Memo `protobuf:"bytes,16,opt,name=memo,proto3" json:"memo,omitempty"` + SearchAttributes *v1.SearchAttributes `protobuf:"bytes,17,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` + // If this is set, the child workflow inherits the Build ID of the parent. Otherwise, the assignment + // rules of the child's Task Queue will be used to independently assign a Build ID to it. + InheritBuildId bool `protobuf:"varint,19,opt,name=inherit_build_id,json=inheritBuildId,proto3" json:"inherit_build_id,omitempty"` +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) Reset() { + *x = StartChildWorkflowExecutionInitiatedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartChildWorkflowExecutionInitiatedEventAttributes) ProtoMessage() {} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[33] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartChildWorkflowExecutionInitiatedEventAttributes.ProtoReflect.Descriptor instead. +func (*StartChildWorkflowExecutionInitiatedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{33} +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetTaskQueue() *v11.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetInput() *v1.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetWorkflowExecutionTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowExecutionTimeout + } + return nil +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetWorkflowRunTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowRunTimeout + } + return nil +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetWorkflowTaskTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowTaskTimeout + } + return nil +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetParentClosePolicy() v12.ParentClosePolicy { + if x != nil { + return x.ParentClosePolicy + } + return v12.ParentClosePolicy(0) +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetControl() string { + if x != nil { + return x.Control + } + return "" +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetWorkflowIdReusePolicy() v12.WorkflowIdReusePolicy { + if x != nil { + return x.WorkflowIdReusePolicy + } + return v12.WorkflowIdReusePolicy(0) +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetRetryPolicy() *v1.RetryPolicy { + if x != nil { + return x.RetryPolicy + } + return nil +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetCronSchedule() string { + if x != nil { + return x.CronSchedule + } + return "" +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetMemo() *v1.Memo { + if x != nil { + return x.Memo + } + return nil +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetSearchAttributes() *v1.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +func (x *StartChildWorkflowExecutionInitiatedEventAttributes) GetInheritBuildId() bool { + if x != nil { + return x.InheritBuildId + } + return false +} + +type StartChildWorkflowExecutionFailedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace of the child workflow. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,8,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowId string `protobuf:"bytes,2,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + WorkflowType *v1.WorkflowType `protobuf:"bytes,3,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + Cause v12.StartChildWorkflowExecutionFailedCause `protobuf:"varint,4,opt,name=cause,proto3,enum=temporal.api.enums.v1.StartChildWorkflowExecutionFailedCause" json:"cause,omitempty"` + // Deprecated + Control string `protobuf:"bytes,5,opt,name=control,proto3" json:"control,omitempty"` + // Id of the `START_CHILD_WORKFLOW_EXECUTION_INITIATED` event which this event corresponds to + InitiatedEventId int64 `protobuf:"varint,6,opt,name=initiated_event_id,json=initiatedEventId,proto3" json:"initiated_event_id,omitempty"` + // The `WORKFLOW_TASK_COMPLETED` event which this command was reported with + WorkflowTaskCompletedEventId int64 `protobuf:"varint,7,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` +} + +func (x *StartChildWorkflowExecutionFailedEventAttributes) Reset() { + *x = StartChildWorkflowExecutionFailedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartChildWorkflowExecutionFailedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartChildWorkflowExecutionFailedEventAttributes) ProtoMessage() {} + +func (x *StartChildWorkflowExecutionFailedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[34] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartChildWorkflowExecutionFailedEventAttributes.ProtoReflect.Descriptor instead. +func (*StartChildWorkflowExecutionFailedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{34} +} + +func (x *StartChildWorkflowExecutionFailedEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *StartChildWorkflowExecutionFailedEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *StartChildWorkflowExecutionFailedEventAttributes) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *StartChildWorkflowExecutionFailedEventAttributes) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *StartChildWorkflowExecutionFailedEventAttributes) GetCause() v12.StartChildWorkflowExecutionFailedCause { + if x != nil { + return x.Cause + } + return v12.StartChildWorkflowExecutionFailedCause(0) +} + +func (x *StartChildWorkflowExecutionFailedEventAttributes) GetControl() string { + if x != nil { + return x.Control + } + return "" +} + +func (x *StartChildWorkflowExecutionFailedEventAttributes) GetInitiatedEventId() int64 { + if x != nil { + return x.InitiatedEventId + } + return 0 +} + +func (x *StartChildWorkflowExecutionFailedEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +type ChildWorkflowExecutionStartedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace of the child workflow. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,6,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + // Id of the `START_CHILD_WORKFLOW_EXECUTION_INITIATED` event which this event corresponds to + InitiatedEventId int64 `protobuf:"varint,2,opt,name=initiated_event_id,json=initiatedEventId,proto3" json:"initiated_event_id,omitempty"` + WorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,3,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + WorkflowType *v1.WorkflowType `protobuf:"bytes,4,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + Header *v1.Header `protobuf:"bytes,5,opt,name=header,proto3" json:"header,omitempty"` +} + +func (x *ChildWorkflowExecutionStartedEventAttributes) Reset() { + *x = ChildWorkflowExecutionStartedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChildWorkflowExecutionStartedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChildWorkflowExecutionStartedEventAttributes) ProtoMessage() {} + +func (x *ChildWorkflowExecutionStartedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[35] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ChildWorkflowExecutionStartedEventAttributes.ProtoReflect.Descriptor instead. +func (*ChildWorkflowExecutionStartedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{35} +} + +func (x *ChildWorkflowExecutionStartedEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ChildWorkflowExecutionStartedEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *ChildWorkflowExecutionStartedEventAttributes) GetInitiatedEventId() int64 { + if x != nil { + return x.InitiatedEventId + } + return 0 +} + +func (x *ChildWorkflowExecutionStartedEventAttributes) GetWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *ChildWorkflowExecutionStartedEventAttributes) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *ChildWorkflowExecutionStartedEventAttributes) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +type ChildWorkflowExecutionCompletedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Result *v1.Payloads `protobuf:"bytes,1,opt,name=result,proto3" json:"result,omitempty"` + // Namespace of the child workflow. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,7,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,3,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + WorkflowType *v1.WorkflowType `protobuf:"bytes,4,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + // Id of the `START_CHILD_WORKFLOW_EXECUTION_INITIATED` event which this event corresponds to + InitiatedEventId int64 `protobuf:"varint,5,opt,name=initiated_event_id,json=initiatedEventId,proto3" json:"initiated_event_id,omitempty"` + // Id of the `CHILD_WORKFLOW_EXECUTION_STARTED` event which this event corresponds to + StartedEventId int64 `protobuf:"varint,6,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` +} + +func (x *ChildWorkflowExecutionCompletedEventAttributes) Reset() { + *x = ChildWorkflowExecutionCompletedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChildWorkflowExecutionCompletedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChildWorkflowExecutionCompletedEventAttributes) ProtoMessage() {} + +func (x *ChildWorkflowExecutionCompletedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[36] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ChildWorkflowExecutionCompletedEventAttributes.ProtoReflect.Descriptor instead. +func (*ChildWorkflowExecutionCompletedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{36} +} + +func (x *ChildWorkflowExecutionCompletedEventAttributes) GetResult() *v1.Payloads { + if x != nil { + return x.Result + } + return nil +} + +func (x *ChildWorkflowExecutionCompletedEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ChildWorkflowExecutionCompletedEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *ChildWorkflowExecutionCompletedEventAttributes) GetWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *ChildWorkflowExecutionCompletedEventAttributes) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *ChildWorkflowExecutionCompletedEventAttributes) GetInitiatedEventId() int64 { + if x != nil { + return x.InitiatedEventId + } + return 0 +} + +func (x *ChildWorkflowExecutionCompletedEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +type ChildWorkflowExecutionFailedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Failure *v13.Failure `protobuf:"bytes,1,opt,name=failure,proto3" json:"failure,omitempty"` + // Namespace of the child workflow. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,8,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,3,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + WorkflowType *v1.WorkflowType `protobuf:"bytes,4,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + // Id of the `START_CHILD_WORKFLOW_EXECUTION_INITIATED` event which this event corresponds to + InitiatedEventId int64 `protobuf:"varint,5,opt,name=initiated_event_id,json=initiatedEventId,proto3" json:"initiated_event_id,omitempty"` + // Id of the `CHILD_WORKFLOW_EXECUTION_STARTED` event which this event corresponds to + StartedEventId int64 `protobuf:"varint,6,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` + RetryState v12.RetryState `protobuf:"varint,7,opt,name=retry_state,json=retryState,proto3,enum=temporal.api.enums.v1.RetryState" json:"retry_state,omitempty"` +} + +func (x *ChildWorkflowExecutionFailedEventAttributes) Reset() { + *x = ChildWorkflowExecutionFailedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChildWorkflowExecutionFailedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChildWorkflowExecutionFailedEventAttributes) ProtoMessage() {} + +func (x *ChildWorkflowExecutionFailedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[37] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ChildWorkflowExecutionFailedEventAttributes.ProtoReflect.Descriptor instead. +func (*ChildWorkflowExecutionFailedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{37} +} + +func (x *ChildWorkflowExecutionFailedEventAttributes) GetFailure() *v13.Failure { + if x != nil { + return x.Failure + } + return nil +} + +func (x *ChildWorkflowExecutionFailedEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ChildWorkflowExecutionFailedEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *ChildWorkflowExecutionFailedEventAttributes) GetWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *ChildWorkflowExecutionFailedEventAttributes) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *ChildWorkflowExecutionFailedEventAttributes) GetInitiatedEventId() int64 { + if x != nil { + return x.InitiatedEventId + } + return 0 +} + +func (x *ChildWorkflowExecutionFailedEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +func (x *ChildWorkflowExecutionFailedEventAttributes) GetRetryState() v12.RetryState { + if x != nil { + return x.RetryState + } + return v12.RetryState(0) +} + +type ChildWorkflowExecutionCanceledEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Details *v1.Payloads `protobuf:"bytes,1,opt,name=details,proto3" json:"details,omitempty"` + // Namespace of the child workflow. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,7,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,3,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + WorkflowType *v1.WorkflowType `protobuf:"bytes,4,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + // Id of the `START_CHILD_WORKFLOW_EXECUTION_INITIATED` event which this event corresponds to + InitiatedEventId int64 `protobuf:"varint,5,opt,name=initiated_event_id,json=initiatedEventId,proto3" json:"initiated_event_id,omitempty"` + // Id of the `CHILD_WORKFLOW_EXECUTION_STARTED` event which this event corresponds to + StartedEventId int64 `protobuf:"varint,6,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` +} + +func (x *ChildWorkflowExecutionCanceledEventAttributes) Reset() { + *x = ChildWorkflowExecutionCanceledEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChildWorkflowExecutionCanceledEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChildWorkflowExecutionCanceledEventAttributes) ProtoMessage() {} + +func (x *ChildWorkflowExecutionCanceledEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[38] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ChildWorkflowExecutionCanceledEventAttributes.ProtoReflect.Descriptor instead. +func (*ChildWorkflowExecutionCanceledEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{38} +} + +func (x *ChildWorkflowExecutionCanceledEventAttributes) GetDetails() *v1.Payloads { + if x != nil { + return x.Details + } + return nil +} + +func (x *ChildWorkflowExecutionCanceledEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ChildWorkflowExecutionCanceledEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *ChildWorkflowExecutionCanceledEventAttributes) GetWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *ChildWorkflowExecutionCanceledEventAttributes) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *ChildWorkflowExecutionCanceledEventAttributes) GetInitiatedEventId() int64 { + if x != nil { + return x.InitiatedEventId + } + return 0 +} + +func (x *ChildWorkflowExecutionCanceledEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +type ChildWorkflowExecutionTimedOutEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace of the child workflow. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,7,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,2,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + WorkflowType *v1.WorkflowType `protobuf:"bytes,3,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + // Id of the `START_CHILD_WORKFLOW_EXECUTION_INITIATED` event which this event corresponds to + InitiatedEventId int64 `protobuf:"varint,4,opt,name=initiated_event_id,json=initiatedEventId,proto3" json:"initiated_event_id,omitempty"` + // Id of the `CHILD_WORKFLOW_EXECUTION_STARTED` event which this event corresponds to + StartedEventId int64 `protobuf:"varint,5,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` + RetryState v12.RetryState `protobuf:"varint,6,opt,name=retry_state,json=retryState,proto3,enum=temporal.api.enums.v1.RetryState" json:"retry_state,omitempty"` +} + +func (x *ChildWorkflowExecutionTimedOutEventAttributes) Reset() { + *x = ChildWorkflowExecutionTimedOutEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChildWorkflowExecutionTimedOutEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChildWorkflowExecutionTimedOutEventAttributes) ProtoMessage() {} + +func (x *ChildWorkflowExecutionTimedOutEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[39] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ChildWorkflowExecutionTimedOutEventAttributes.ProtoReflect.Descriptor instead. +func (*ChildWorkflowExecutionTimedOutEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{39} +} + +func (x *ChildWorkflowExecutionTimedOutEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ChildWorkflowExecutionTimedOutEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *ChildWorkflowExecutionTimedOutEventAttributes) GetWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *ChildWorkflowExecutionTimedOutEventAttributes) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *ChildWorkflowExecutionTimedOutEventAttributes) GetInitiatedEventId() int64 { + if x != nil { + return x.InitiatedEventId + } + return 0 +} + +func (x *ChildWorkflowExecutionTimedOutEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +func (x *ChildWorkflowExecutionTimedOutEventAttributes) GetRetryState() v12.RetryState { + if x != nil { + return x.RetryState + } + return v12.RetryState(0) +} + +type ChildWorkflowExecutionTerminatedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace of the child workflow. + // SDKs and UI tools should use `namespace` field but server must use `namespace_id` only. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,6,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + WorkflowExecution *v1.WorkflowExecution `protobuf:"bytes,2,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + WorkflowType *v1.WorkflowType `protobuf:"bytes,3,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + // Id of the `START_CHILD_WORKFLOW_EXECUTION_INITIATED` event which this event corresponds to + InitiatedEventId int64 `protobuf:"varint,4,opt,name=initiated_event_id,json=initiatedEventId,proto3" json:"initiated_event_id,omitempty"` + // Id of the `CHILD_WORKFLOW_EXECUTION_STARTED` event which this event corresponds to + StartedEventId int64 `protobuf:"varint,5,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` +} + +func (x *ChildWorkflowExecutionTerminatedEventAttributes) Reset() { + *x = ChildWorkflowExecutionTerminatedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ChildWorkflowExecutionTerminatedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChildWorkflowExecutionTerminatedEventAttributes) ProtoMessage() {} + +func (x *ChildWorkflowExecutionTerminatedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[40] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ChildWorkflowExecutionTerminatedEventAttributes.ProtoReflect.Descriptor instead. +func (*ChildWorkflowExecutionTerminatedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{40} +} + +func (x *ChildWorkflowExecutionTerminatedEventAttributes) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ChildWorkflowExecutionTerminatedEventAttributes) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *ChildWorkflowExecutionTerminatedEventAttributes) GetWorkflowExecution() *v1.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *ChildWorkflowExecutionTerminatedEventAttributes) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *ChildWorkflowExecutionTerminatedEventAttributes) GetInitiatedEventId() int64 { + if x != nil { + return x.InitiatedEventId + } + return 0 +} + +func (x *ChildWorkflowExecutionTerminatedEventAttributes) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +type WorkflowPropertiesModifiedExternallyEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // If set to a nonempty string, future workflow tasks for this workflow shall be dispatched on + // the provided queue. + NewTaskQueue string `protobuf:"bytes,1,opt,name=new_task_queue,json=newTaskQueue,proto3" json:"new_task_queue,omitempty"` + // If set, update the workflow task timeout to this value. + NewWorkflowTaskTimeout *durationpb.Duration `protobuf:"bytes,2,opt,name=new_workflow_task_timeout,json=newWorkflowTaskTimeout,proto3" json:"new_workflow_task_timeout,omitempty"` + // If set, update the workflow run timeout to this value. May be set to 0 for no timeout. + NewWorkflowRunTimeout *durationpb.Duration `protobuf:"bytes,3,opt,name=new_workflow_run_timeout,json=newWorkflowRunTimeout,proto3" json:"new_workflow_run_timeout,omitempty"` + // If set, update the workflow execution timeout to this value. May be set to 0 for no timeout. + NewWorkflowExecutionTimeout *durationpb.Duration `protobuf:"bytes,4,opt,name=new_workflow_execution_timeout,json=newWorkflowExecutionTimeout,proto3" json:"new_workflow_execution_timeout,omitempty"` + // If set, update the workflow memo with the provided values. The values will be merged with + // the existing memo. If the user wants to delete values, a default/empty Payload should be + // used as the value for the key being deleted. + UpsertedMemo *v1.Memo `protobuf:"bytes,5,opt,name=upserted_memo,json=upsertedMemo,proto3" json:"upserted_memo,omitempty"` +} + +func (x *WorkflowPropertiesModifiedExternallyEventAttributes) Reset() { + *x = WorkflowPropertiesModifiedExternallyEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowPropertiesModifiedExternallyEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowPropertiesModifiedExternallyEventAttributes) ProtoMessage() {} + +func (x *WorkflowPropertiesModifiedExternallyEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[41] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowPropertiesModifiedExternallyEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowPropertiesModifiedExternallyEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{41} +} + +func (x *WorkflowPropertiesModifiedExternallyEventAttributes) GetNewTaskQueue() string { + if x != nil { + return x.NewTaskQueue + } + return "" +} + +func (x *WorkflowPropertiesModifiedExternallyEventAttributes) GetNewWorkflowTaskTimeout() *durationpb.Duration { + if x != nil { + return x.NewWorkflowTaskTimeout + } + return nil +} + +func (x *WorkflowPropertiesModifiedExternallyEventAttributes) GetNewWorkflowRunTimeout() *durationpb.Duration { + if x != nil { + return x.NewWorkflowRunTimeout + } + return nil +} + +func (x *WorkflowPropertiesModifiedExternallyEventAttributes) GetNewWorkflowExecutionTimeout() *durationpb.Duration { + if x != nil { + return x.NewWorkflowExecutionTimeout + } + return nil +} + +func (x *WorkflowPropertiesModifiedExternallyEventAttributes) GetUpsertedMemo() *v1.Memo { + if x != nil { + return x.UpsertedMemo + } + return nil +} + +type ActivityPropertiesModifiedExternallyEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the `ACTIVITY_TASK_SCHEDULED` event this modification corresponds to. + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // If set, update the retry policy of the activity, replacing it with the specified one. + // The number of attempts at the activity is preserved. + NewRetryPolicy *v1.RetryPolicy `protobuf:"bytes,2,opt,name=new_retry_policy,json=newRetryPolicy,proto3" json:"new_retry_policy,omitempty"` +} + +func (x *ActivityPropertiesModifiedExternallyEventAttributes) Reset() { + *x = ActivityPropertiesModifiedExternallyEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActivityPropertiesModifiedExternallyEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActivityPropertiesModifiedExternallyEventAttributes) ProtoMessage() {} + +func (x *ActivityPropertiesModifiedExternallyEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[42] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActivityPropertiesModifiedExternallyEventAttributes.ProtoReflect.Descriptor instead. +func (*ActivityPropertiesModifiedExternallyEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{42} +} + +func (x *ActivityPropertiesModifiedExternallyEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *ActivityPropertiesModifiedExternallyEventAttributes) GetNewRetryPolicy() *v1.RetryPolicy { + if x != nil { + return x.NewRetryPolicy + } + return nil +} + +type WorkflowExecutionUpdateAcceptedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The instance ID of the update protocol that generated this event. + ProtocolInstanceId string `protobuf:"bytes,1,opt,name=protocol_instance_id,json=protocolInstanceId,proto3" json:"protocol_instance_id,omitempty"` + // The message ID of the original request message that initiated this + // update. Needed so that the worker can recreate and deliver that same + // message as part of replay. + AcceptedRequestMessageId string `protobuf:"bytes,2,opt,name=accepted_request_message_id,json=acceptedRequestMessageId,proto3" json:"accepted_request_message_id,omitempty"` + // The event ID used to sequence the original request message. + AcceptedRequestSequencingEventId int64 `protobuf:"varint,3,opt,name=accepted_request_sequencing_event_id,json=acceptedRequestSequencingEventId,proto3" json:"accepted_request_sequencing_event_id,omitempty"` + // The message payload of the original request message that initiated this + // update. + AcceptedRequest *v16.Request `protobuf:"bytes,4,opt,name=accepted_request,json=acceptedRequest,proto3" json:"accepted_request,omitempty"` +} + +func (x *WorkflowExecutionUpdateAcceptedEventAttributes) Reset() { + *x = WorkflowExecutionUpdateAcceptedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionUpdateAcceptedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionUpdateAcceptedEventAttributes) ProtoMessage() {} + +func (x *WorkflowExecutionUpdateAcceptedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[43] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionUpdateAcceptedEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionUpdateAcceptedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{43} +} + +func (x *WorkflowExecutionUpdateAcceptedEventAttributes) GetProtocolInstanceId() string { + if x != nil { + return x.ProtocolInstanceId + } + return "" +} + +func (x *WorkflowExecutionUpdateAcceptedEventAttributes) GetAcceptedRequestMessageId() string { + if x != nil { + return x.AcceptedRequestMessageId + } + return "" +} + +func (x *WorkflowExecutionUpdateAcceptedEventAttributes) GetAcceptedRequestSequencingEventId() int64 { + if x != nil { + return x.AcceptedRequestSequencingEventId + } + return 0 +} + +func (x *WorkflowExecutionUpdateAcceptedEventAttributes) GetAcceptedRequest() *v16.Request { + if x != nil { + return x.AcceptedRequest + } + return nil +} + +type WorkflowExecutionUpdateCompletedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The metadata about this update. + Meta *v16.Meta `protobuf:"bytes,1,opt,name=meta,proto3" json:"meta,omitempty"` + // The event ID indicating the acceptance of this update. + AcceptedEventId int64 `protobuf:"varint,3,opt,name=accepted_event_id,json=acceptedEventId,proto3" json:"accepted_event_id,omitempty"` + // The outcome of executing the workflow update function. + Outcome *v16.Outcome `protobuf:"bytes,2,opt,name=outcome,proto3" json:"outcome,omitempty"` +} + +func (x *WorkflowExecutionUpdateCompletedEventAttributes) Reset() { + *x = WorkflowExecutionUpdateCompletedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionUpdateCompletedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionUpdateCompletedEventAttributes) ProtoMessage() {} + +func (x *WorkflowExecutionUpdateCompletedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[44] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionUpdateCompletedEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionUpdateCompletedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{44} +} + +func (x *WorkflowExecutionUpdateCompletedEventAttributes) GetMeta() *v16.Meta { + if x != nil { + return x.Meta + } + return nil +} + +func (x *WorkflowExecutionUpdateCompletedEventAttributes) GetAcceptedEventId() int64 { + if x != nil { + return x.AcceptedEventId + } + return 0 +} + +func (x *WorkflowExecutionUpdateCompletedEventAttributes) GetOutcome() *v16.Outcome { + if x != nil { + return x.Outcome + } + return nil +} + +type WorkflowExecutionUpdateRejectedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The instance ID of the update protocol that generated this event. + ProtocolInstanceId string `protobuf:"bytes,1,opt,name=protocol_instance_id,json=protocolInstanceId,proto3" json:"protocol_instance_id,omitempty"` + // The message ID of the original request message that initiated this + // update. Needed so that the worker can recreate and deliver that same + // message as part of replay. + RejectedRequestMessageId string `protobuf:"bytes,2,opt,name=rejected_request_message_id,json=rejectedRequestMessageId,proto3" json:"rejected_request_message_id,omitempty"` + // The event ID used to sequence the original request message. + RejectedRequestSequencingEventId int64 `protobuf:"varint,3,opt,name=rejected_request_sequencing_event_id,json=rejectedRequestSequencingEventId,proto3" json:"rejected_request_sequencing_event_id,omitempty"` + // The message payload of the original request message that initiated this + // update. + RejectedRequest *v16.Request `protobuf:"bytes,4,opt,name=rejected_request,json=rejectedRequest,proto3" json:"rejected_request,omitempty"` + // The cause of rejection. + Failure *v13.Failure `protobuf:"bytes,5,opt,name=failure,proto3" json:"failure,omitempty"` +} + +func (x *WorkflowExecutionUpdateRejectedEventAttributes) Reset() { + *x = WorkflowExecutionUpdateRejectedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionUpdateRejectedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionUpdateRejectedEventAttributes) ProtoMessage() {} + +func (x *WorkflowExecutionUpdateRejectedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[45] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionUpdateRejectedEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionUpdateRejectedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{45} +} + +func (x *WorkflowExecutionUpdateRejectedEventAttributes) GetProtocolInstanceId() string { + if x != nil { + return x.ProtocolInstanceId + } + return "" +} + +func (x *WorkflowExecutionUpdateRejectedEventAttributes) GetRejectedRequestMessageId() string { + if x != nil { + return x.RejectedRequestMessageId + } + return "" +} + +func (x *WorkflowExecutionUpdateRejectedEventAttributes) GetRejectedRequestSequencingEventId() int64 { + if x != nil { + return x.RejectedRequestSequencingEventId + } + return 0 +} + +func (x *WorkflowExecutionUpdateRejectedEventAttributes) GetRejectedRequest() *v16.Request { + if x != nil { + return x.RejectedRequest + } + return nil +} + +func (x *WorkflowExecutionUpdateRejectedEventAttributes) GetFailure() *v13.Failure { + if x != nil { + return x.Failure + } + return nil +} + +type WorkflowExecutionUpdateAdmittedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The update request associated with this event. + Request *v16.Request `protobuf:"bytes,1,opt,name=request,proto3" json:"request,omitempty"` + // An explanation of why this event was written to history. + Origin v12.UpdateAdmittedEventOrigin `protobuf:"varint,2,opt,name=origin,proto3,enum=temporal.api.enums.v1.UpdateAdmittedEventOrigin" json:"origin,omitempty"` +} + +func (x *WorkflowExecutionUpdateAdmittedEventAttributes) Reset() { + *x = WorkflowExecutionUpdateAdmittedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[46] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionUpdateAdmittedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionUpdateAdmittedEventAttributes) ProtoMessage() {} + +func (x *WorkflowExecutionUpdateAdmittedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[46] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionUpdateAdmittedEventAttributes.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionUpdateAdmittedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{46} +} + +func (x *WorkflowExecutionUpdateAdmittedEventAttributes) GetRequest() *v16.Request { + if x != nil { + return x.Request + } + return nil +} + +func (x *WorkflowExecutionUpdateAdmittedEventAttributes) GetOrigin() v12.UpdateAdmittedEventOrigin { + if x != nil { + return x.Origin + } + return v12.UpdateAdmittedEventOrigin(0) +} + +// Event marking that an operation was scheduled by a workflow via the ScheduleNexusOperation command. +type NexusOperationScheduledEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Endpoint name, must exist in the endpoint registry. + Endpoint string `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + // Service name. + Service string `protobuf:"bytes,2,opt,name=service,proto3" json:"service,omitempty"` + // Operation name. + Operation string `protobuf:"bytes,3,opt,name=operation,proto3" json:"operation,omitempty"` + // Input for the operation. The server converts this into Nexus request content and the appropriate content headers + // internally when sending the StartOperation request. On the handler side, if it is also backed by Temporal, the + // content is transformed back to the original Payload stored in this event. + Input *v1.Payload `protobuf:"bytes,4,opt,name=input,proto3" json:"input,omitempty"` + // Schedule-to-close timeout for this operation. + // Indicates how long the caller is willing to wait for operation completion. + // Calls are retried internally by the server. + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + ScheduleToCloseTimeout *durationpb.Duration `protobuf:"bytes,5,opt,name=schedule_to_close_timeout,json=scheduleToCloseTimeout,proto3" json:"schedule_to_close_timeout,omitempty"` + // Header to attach to the Nexus request. Note these headers are not the same as Temporal headers on internal + // activities and child workflows, these are transmitted to Nexus operations that may be external and are not + // traditional payloads. + NexusHeader map[string]string `protobuf:"bytes,6,rep,name=nexus_header,json=nexusHeader,proto3" json:"nexus_header,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // The `WORKFLOW_TASK_COMPLETED` event that the corresponding ScheduleNexusOperation command was reported with. + WorkflowTaskCompletedEventId int64 `protobuf:"varint,7,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` + // A unique ID generated by the history service upon creation of this event. + // The ID will be transmitted with all nexus StartOperation requests and is used as an idempotentency key. + RequestId string `protobuf:"bytes,8,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` + // Endpoint ID as resolved in the endpoint registry at the time this event was generated. + // This is stored on the event and used internally by the server in case the endpoint is renamed from the time the + // event was originally scheduled. + EndpointId string `protobuf:"bytes,9,opt,name=endpoint_id,json=endpointId,proto3" json:"endpoint_id,omitempty"` +} + +func (x *NexusOperationScheduledEventAttributes) Reset() { + *x = NexusOperationScheduledEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[47] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NexusOperationScheduledEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NexusOperationScheduledEventAttributes) ProtoMessage() {} + +func (x *NexusOperationScheduledEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[47] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NexusOperationScheduledEventAttributes.ProtoReflect.Descriptor instead. +func (*NexusOperationScheduledEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{47} +} + +func (x *NexusOperationScheduledEventAttributes) GetEndpoint() string { + if x != nil { + return x.Endpoint + } + return "" +} + +func (x *NexusOperationScheduledEventAttributes) GetService() string { + if x != nil { + return x.Service + } + return "" +} + +func (x *NexusOperationScheduledEventAttributes) GetOperation() string { + if x != nil { + return x.Operation + } + return "" +} + +func (x *NexusOperationScheduledEventAttributes) GetInput() *v1.Payload { + if x != nil { + return x.Input + } + return nil +} + +func (x *NexusOperationScheduledEventAttributes) GetScheduleToCloseTimeout() *durationpb.Duration { + if x != nil { + return x.ScheduleToCloseTimeout + } + return nil +} + +func (x *NexusOperationScheduledEventAttributes) GetNexusHeader() map[string]string { + if x != nil { + return x.NexusHeader + } + return nil +} + +func (x *NexusOperationScheduledEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +func (x *NexusOperationScheduledEventAttributes) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +func (x *NexusOperationScheduledEventAttributes) GetEndpointId() string { + if x != nil { + return x.EndpointId + } + return "" +} + +// Event marking an asynchronous operation was started by the responding Nexus handler. +// If the operation completes synchronously, this event is not generated. +// In rare situations, such as request timeouts, the service may fail to record the actual start time and will fabricate +// this event upon receiving the operation completion via callback. +type NexusOperationStartedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The ID of the `NEXUS_OPERATION_SCHEDULED` event this task corresponds to. + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // The operation ID returned by the Nexus handler in the response to the StartOperation request. + // This ID is used when canceling the operation. + OperationId string `protobuf:"bytes,3,opt,name=operation_id,json=operationId,proto3" json:"operation_id,omitempty"` + // The request ID allocated at schedule time. + RequestId string `protobuf:"bytes,4,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` +} + +func (x *NexusOperationStartedEventAttributes) Reset() { + *x = NexusOperationStartedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[48] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NexusOperationStartedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NexusOperationStartedEventAttributes) ProtoMessage() {} + +func (x *NexusOperationStartedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[48] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NexusOperationStartedEventAttributes.ProtoReflect.Descriptor instead. +func (*NexusOperationStartedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{48} +} + +func (x *NexusOperationStartedEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *NexusOperationStartedEventAttributes) GetOperationId() string { + if x != nil { + return x.OperationId + } + return "" +} + +func (x *NexusOperationStartedEventAttributes) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +// Nexus operation completed successfully. +type NexusOperationCompletedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The ID of the `NEXUS_OPERATION_SCHEDULED` event. Uniquely identifies this operation. + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // Serialized result of the Nexus operation. The response of the Nexus handler. + // Delivered either via a completion callback or as a response to a synchronous operation. + Result *v1.Payload `protobuf:"bytes,2,opt,name=result,proto3" json:"result,omitempty"` + // The request ID allocated at schedule time. + RequestId string `protobuf:"bytes,3,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` +} + +func (x *NexusOperationCompletedEventAttributes) Reset() { + *x = NexusOperationCompletedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[49] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NexusOperationCompletedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NexusOperationCompletedEventAttributes) ProtoMessage() {} + +func (x *NexusOperationCompletedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[49] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NexusOperationCompletedEventAttributes.ProtoReflect.Descriptor instead. +func (*NexusOperationCompletedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{49} +} + +func (x *NexusOperationCompletedEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *NexusOperationCompletedEventAttributes) GetResult() *v1.Payload { + if x != nil { + return x.Result + } + return nil +} + +func (x *NexusOperationCompletedEventAttributes) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +// Nexus operation failed. +type NexusOperationFailedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The ID of the `NEXUS_OPERATION_SCHEDULED` event. Uniquely identifies this operation. + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // Failure details. A NexusOperationFailureInfo wrapping an ApplicationFailureInfo. + Failure *v13.Failure `protobuf:"bytes,2,opt,name=failure,proto3" json:"failure,omitempty"` + // The request ID allocated at schedule time. + RequestId string `protobuf:"bytes,3,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` +} + +func (x *NexusOperationFailedEventAttributes) Reset() { + *x = NexusOperationFailedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[50] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NexusOperationFailedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NexusOperationFailedEventAttributes) ProtoMessage() {} + +func (x *NexusOperationFailedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[50] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NexusOperationFailedEventAttributes.ProtoReflect.Descriptor instead. +func (*NexusOperationFailedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{50} +} + +func (x *NexusOperationFailedEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *NexusOperationFailedEventAttributes) GetFailure() *v13.Failure { + if x != nil { + return x.Failure + } + return nil +} + +func (x *NexusOperationFailedEventAttributes) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +// Nexus operation timed out. +type NexusOperationTimedOutEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The ID of the `NEXUS_OPERATION_SCHEDULED` event. Uniquely identifies this operation. + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // Failure details. A NexusOperationFailureInfo wrapping a CanceledFailureInfo. + Failure *v13.Failure `protobuf:"bytes,2,opt,name=failure,proto3" json:"failure,omitempty"` + // The request ID allocated at schedule time. + RequestId string `protobuf:"bytes,3,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` +} + +func (x *NexusOperationTimedOutEventAttributes) Reset() { + *x = NexusOperationTimedOutEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[51] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NexusOperationTimedOutEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NexusOperationTimedOutEventAttributes) ProtoMessage() {} + +func (x *NexusOperationTimedOutEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[51] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NexusOperationTimedOutEventAttributes.ProtoReflect.Descriptor instead. +func (*NexusOperationTimedOutEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{51} +} + +func (x *NexusOperationTimedOutEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *NexusOperationTimedOutEventAttributes) GetFailure() *v13.Failure { + if x != nil { + return x.Failure + } + return nil +} + +func (x *NexusOperationTimedOutEventAttributes) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +// Nexus operation completed as canceled. May or may not have been due to a cancellation request by the workflow. +type NexusOperationCanceledEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The ID of the `NEXUS_OPERATION_SCHEDULED` event. Uniquely identifies this operation. + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // Cancellation details. + Failure *v13.Failure `protobuf:"bytes,2,opt,name=failure,proto3" json:"failure,omitempty"` + // The request ID allocated at schedule time. + RequestId string `protobuf:"bytes,3,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` +} + +func (x *NexusOperationCanceledEventAttributes) Reset() { + *x = NexusOperationCanceledEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[52] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NexusOperationCanceledEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NexusOperationCanceledEventAttributes) ProtoMessage() {} + +func (x *NexusOperationCanceledEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[52] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NexusOperationCanceledEventAttributes.ProtoReflect.Descriptor instead. +func (*NexusOperationCanceledEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{52} +} + +func (x *NexusOperationCanceledEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *NexusOperationCanceledEventAttributes) GetFailure() *v13.Failure { + if x != nil { + return x.Failure + } + return nil +} + +func (x *NexusOperationCanceledEventAttributes) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +type NexusOperationCancelRequestedEventAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The id of the `NEXUS_OPERATION_SCHEDULED` event this cancel request corresponds to. + ScheduledEventId int64 `protobuf:"varint,1,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` + // The `WORKFLOW_TASK_COMPLETED` event that the corresponding RequestCancelNexusOperation command was reported + // with. + WorkflowTaskCompletedEventId int64 `protobuf:"varint,2,opt,name=workflow_task_completed_event_id,json=workflowTaskCompletedEventId,proto3" json:"workflow_task_completed_event_id,omitempty"` +} + +func (x *NexusOperationCancelRequestedEventAttributes) Reset() { + *x = NexusOperationCancelRequestedEventAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[53] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NexusOperationCancelRequestedEventAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NexusOperationCancelRequestedEventAttributes) ProtoMessage() {} + +func (x *NexusOperationCancelRequestedEventAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[53] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NexusOperationCancelRequestedEventAttributes.ProtoReflect.Descriptor instead. +func (*NexusOperationCancelRequestedEventAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{53} +} + +func (x *NexusOperationCancelRequestedEventAttributes) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +func (x *NexusOperationCancelRequestedEventAttributes) GetWorkflowTaskCompletedEventId() int64 { + if x != nil { + return x.WorkflowTaskCompletedEventId + } + return 0 +} + +// History events are the method by which Temporal SDKs advance (or recreate) workflow state. +// See the `EventType` enum for more info about what each event is for. +type HistoryEvent struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Monotonically increasing event number, starts at 1. + EventId int64 `protobuf:"varint,1,opt,name=event_id,json=eventId,proto3" json:"event_id,omitempty"` + EventTime *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=event_time,json=eventTime,proto3" json:"event_time,omitempty"` + EventType v12.EventType `protobuf:"varint,3,opt,name=event_type,json=eventType,proto3,enum=temporal.api.enums.v1.EventType" json:"event_type,omitempty"` + // TODO: What is this? Appears unused by SDKs + Version int64 `protobuf:"varint,4,opt,name=version,proto3" json:"version,omitempty"` + // TODO: What is this? Appears unused by SDKs + TaskId int64 `protobuf:"varint,5,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"` + // Set to true when the SDK may ignore the event as it does not impact workflow state or + // information in any way that the SDK need be concerned with. If an SDK encounters an event + // type which it does not understand, it must error unless this is true. If it is true, it's + // acceptable for the event type and/or attributes to be uninterpretable. + WorkerMayIgnore bool `protobuf:"varint,300,opt,name=worker_may_ignore,json=workerMayIgnore,proto3" json:"worker_may_ignore,omitempty"` + // Metadata on the event. This is often carried over from commands and client calls. Most events + // won't have this information, and how this information is used is dependent upon the interface + // that reads it. + // + // Current well-known uses: + // - workflow_execution_started_event_attributes - summary and details from start workflow. + // - timer_started_event_attributes - summary represents an identifier for the timer for use by + // user interfaces. + UserMetadata *v15.UserMetadata `protobuf:"bytes,301,opt,name=user_metadata,json=userMetadata,proto3" json:"user_metadata,omitempty"` + // Links associated with the event. + Links []*v1.Link `protobuf:"bytes,302,rep,name=links,proto3" json:"links,omitempty"` + // The event details. The type must match that in `event_type`. + // + // Types that are assignable to Attributes: + // + // *HistoryEvent_WorkflowExecutionStartedEventAttributes + // *HistoryEvent_WorkflowExecutionCompletedEventAttributes + // *HistoryEvent_WorkflowExecutionFailedEventAttributes + // *HistoryEvent_WorkflowExecutionTimedOutEventAttributes + // *HistoryEvent_WorkflowTaskScheduledEventAttributes + // *HistoryEvent_WorkflowTaskStartedEventAttributes + // *HistoryEvent_WorkflowTaskCompletedEventAttributes + // *HistoryEvent_WorkflowTaskTimedOutEventAttributes + // *HistoryEvent_WorkflowTaskFailedEventAttributes + // *HistoryEvent_ActivityTaskScheduledEventAttributes + // *HistoryEvent_ActivityTaskStartedEventAttributes + // *HistoryEvent_ActivityTaskCompletedEventAttributes + // *HistoryEvent_ActivityTaskFailedEventAttributes + // *HistoryEvent_ActivityTaskTimedOutEventAttributes + // *HistoryEvent_TimerStartedEventAttributes + // *HistoryEvent_TimerFiredEventAttributes + // *HistoryEvent_ActivityTaskCancelRequestedEventAttributes + // *HistoryEvent_ActivityTaskCanceledEventAttributes + // *HistoryEvent_TimerCanceledEventAttributes + // *HistoryEvent_MarkerRecordedEventAttributes + // *HistoryEvent_WorkflowExecutionSignaledEventAttributes + // *HistoryEvent_WorkflowExecutionTerminatedEventAttributes + // *HistoryEvent_WorkflowExecutionCancelRequestedEventAttributes + // *HistoryEvent_WorkflowExecutionCanceledEventAttributes + // *HistoryEvent_RequestCancelExternalWorkflowExecutionInitiatedEventAttributes + // *HistoryEvent_RequestCancelExternalWorkflowExecutionFailedEventAttributes + // *HistoryEvent_ExternalWorkflowExecutionCancelRequestedEventAttributes + // *HistoryEvent_WorkflowExecutionContinuedAsNewEventAttributes + // *HistoryEvent_StartChildWorkflowExecutionInitiatedEventAttributes + // *HistoryEvent_StartChildWorkflowExecutionFailedEventAttributes + // *HistoryEvent_ChildWorkflowExecutionStartedEventAttributes + // *HistoryEvent_ChildWorkflowExecutionCompletedEventAttributes + // *HistoryEvent_ChildWorkflowExecutionFailedEventAttributes + // *HistoryEvent_ChildWorkflowExecutionCanceledEventAttributes + // *HistoryEvent_ChildWorkflowExecutionTimedOutEventAttributes + // *HistoryEvent_ChildWorkflowExecutionTerminatedEventAttributes + // *HistoryEvent_SignalExternalWorkflowExecutionInitiatedEventAttributes + // *HistoryEvent_SignalExternalWorkflowExecutionFailedEventAttributes + // *HistoryEvent_ExternalWorkflowExecutionSignaledEventAttributes + // *HistoryEvent_UpsertWorkflowSearchAttributesEventAttributes + // *HistoryEvent_WorkflowExecutionUpdateAcceptedEventAttributes + // *HistoryEvent_WorkflowExecutionUpdateRejectedEventAttributes + // *HistoryEvent_WorkflowExecutionUpdateCompletedEventAttributes + // *HistoryEvent_WorkflowPropertiesModifiedExternallyEventAttributes + // *HistoryEvent_ActivityPropertiesModifiedExternallyEventAttributes + // *HistoryEvent_WorkflowPropertiesModifiedEventAttributes + // *HistoryEvent_WorkflowExecutionUpdateAdmittedEventAttributes + // *HistoryEvent_NexusOperationScheduledEventAttributes + // *HistoryEvent_NexusOperationStartedEventAttributes + // *HistoryEvent_NexusOperationCompletedEventAttributes + // *HistoryEvent_NexusOperationFailedEventAttributes + // *HistoryEvent_NexusOperationCanceledEventAttributes + // *HistoryEvent_NexusOperationTimedOutEventAttributes + // *HistoryEvent_NexusOperationCancelRequestedEventAttributes + Attributes isHistoryEvent_Attributes `protobuf_oneof:"attributes"` +} + +func (x *HistoryEvent) Reset() { + *x = HistoryEvent{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[54] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *HistoryEvent) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HistoryEvent) ProtoMessage() {} + +func (x *HistoryEvent) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[54] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use HistoryEvent.ProtoReflect.Descriptor instead. +func (*HistoryEvent) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{54} +} + +func (x *HistoryEvent) GetEventId() int64 { + if x != nil { + return x.EventId + } + return 0 +} + +func (x *HistoryEvent) GetEventTime() *timestamppb.Timestamp { + if x != nil { + return x.EventTime + } + return nil +} + +func (x *HistoryEvent) GetEventType() v12.EventType { + if x != nil { + return x.EventType + } + return v12.EventType(0) +} + +func (x *HistoryEvent) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *HistoryEvent) GetTaskId() int64 { + if x != nil { + return x.TaskId + } + return 0 +} + +func (x *HistoryEvent) GetWorkerMayIgnore() bool { + if x != nil { + return x.WorkerMayIgnore + } + return false +} + +func (x *HistoryEvent) GetUserMetadata() *v15.UserMetadata { + if x != nil { + return x.UserMetadata + } + return nil +} + +func (x *HistoryEvent) GetLinks() []*v1.Link { + if x != nil { + return x.Links + } + return nil +} + +func (m *HistoryEvent) GetAttributes() isHistoryEvent_Attributes { + if m != nil { + return m.Attributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowExecutionStartedEventAttributes() *WorkflowExecutionStartedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowExecutionStartedEventAttributes); ok { + return x.WorkflowExecutionStartedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowExecutionCompletedEventAttributes() *WorkflowExecutionCompletedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowExecutionCompletedEventAttributes); ok { + return x.WorkflowExecutionCompletedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowExecutionFailedEventAttributes() *WorkflowExecutionFailedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowExecutionFailedEventAttributes); ok { + return x.WorkflowExecutionFailedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowExecutionTimedOutEventAttributes() *WorkflowExecutionTimedOutEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowExecutionTimedOutEventAttributes); ok { + return x.WorkflowExecutionTimedOutEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowTaskScheduledEventAttributes() *WorkflowTaskScheduledEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowTaskScheduledEventAttributes); ok { + return x.WorkflowTaskScheduledEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowTaskStartedEventAttributes() *WorkflowTaskStartedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowTaskStartedEventAttributes); ok { + return x.WorkflowTaskStartedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowTaskCompletedEventAttributes() *WorkflowTaskCompletedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowTaskCompletedEventAttributes); ok { + return x.WorkflowTaskCompletedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowTaskTimedOutEventAttributes() *WorkflowTaskTimedOutEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowTaskTimedOutEventAttributes); ok { + return x.WorkflowTaskTimedOutEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowTaskFailedEventAttributes() *WorkflowTaskFailedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowTaskFailedEventAttributes); ok { + return x.WorkflowTaskFailedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetActivityTaskScheduledEventAttributes() *ActivityTaskScheduledEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ActivityTaskScheduledEventAttributes); ok { + return x.ActivityTaskScheduledEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetActivityTaskStartedEventAttributes() *ActivityTaskStartedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ActivityTaskStartedEventAttributes); ok { + return x.ActivityTaskStartedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetActivityTaskCompletedEventAttributes() *ActivityTaskCompletedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ActivityTaskCompletedEventAttributes); ok { + return x.ActivityTaskCompletedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetActivityTaskFailedEventAttributes() *ActivityTaskFailedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ActivityTaskFailedEventAttributes); ok { + return x.ActivityTaskFailedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetActivityTaskTimedOutEventAttributes() *ActivityTaskTimedOutEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ActivityTaskTimedOutEventAttributes); ok { + return x.ActivityTaskTimedOutEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetTimerStartedEventAttributes() *TimerStartedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_TimerStartedEventAttributes); ok { + return x.TimerStartedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetTimerFiredEventAttributes() *TimerFiredEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_TimerFiredEventAttributes); ok { + return x.TimerFiredEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetActivityTaskCancelRequestedEventAttributes() *ActivityTaskCancelRequestedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ActivityTaskCancelRequestedEventAttributes); ok { + return x.ActivityTaskCancelRequestedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetActivityTaskCanceledEventAttributes() *ActivityTaskCanceledEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ActivityTaskCanceledEventAttributes); ok { + return x.ActivityTaskCanceledEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetTimerCanceledEventAttributes() *TimerCanceledEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_TimerCanceledEventAttributes); ok { + return x.TimerCanceledEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetMarkerRecordedEventAttributes() *MarkerRecordedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_MarkerRecordedEventAttributes); ok { + return x.MarkerRecordedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowExecutionSignaledEventAttributes() *WorkflowExecutionSignaledEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowExecutionSignaledEventAttributes); ok { + return x.WorkflowExecutionSignaledEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowExecutionTerminatedEventAttributes() *WorkflowExecutionTerminatedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowExecutionTerminatedEventAttributes); ok { + return x.WorkflowExecutionTerminatedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowExecutionCancelRequestedEventAttributes() *WorkflowExecutionCancelRequestedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowExecutionCancelRequestedEventAttributes); ok { + return x.WorkflowExecutionCancelRequestedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowExecutionCanceledEventAttributes() *WorkflowExecutionCanceledEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowExecutionCanceledEventAttributes); ok { + return x.WorkflowExecutionCanceledEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetRequestCancelExternalWorkflowExecutionInitiatedEventAttributes() *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_RequestCancelExternalWorkflowExecutionInitiatedEventAttributes); ok { + return x.RequestCancelExternalWorkflowExecutionInitiatedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetRequestCancelExternalWorkflowExecutionFailedEventAttributes() *RequestCancelExternalWorkflowExecutionFailedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_RequestCancelExternalWorkflowExecutionFailedEventAttributes); ok { + return x.RequestCancelExternalWorkflowExecutionFailedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetExternalWorkflowExecutionCancelRequestedEventAttributes() *ExternalWorkflowExecutionCancelRequestedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ExternalWorkflowExecutionCancelRequestedEventAttributes); ok { + return x.ExternalWorkflowExecutionCancelRequestedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowExecutionContinuedAsNewEventAttributes() *WorkflowExecutionContinuedAsNewEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowExecutionContinuedAsNewEventAttributes); ok { + return x.WorkflowExecutionContinuedAsNewEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetStartChildWorkflowExecutionInitiatedEventAttributes() *StartChildWorkflowExecutionInitiatedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_StartChildWorkflowExecutionInitiatedEventAttributes); ok { + return x.StartChildWorkflowExecutionInitiatedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetStartChildWorkflowExecutionFailedEventAttributes() *StartChildWorkflowExecutionFailedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_StartChildWorkflowExecutionFailedEventAttributes); ok { + return x.StartChildWorkflowExecutionFailedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetChildWorkflowExecutionStartedEventAttributes() *ChildWorkflowExecutionStartedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ChildWorkflowExecutionStartedEventAttributes); ok { + return x.ChildWorkflowExecutionStartedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetChildWorkflowExecutionCompletedEventAttributes() *ChildWorkflowExecutionCompletedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ChildWorkflowExecutionCompletedEventAttributes); ok { + return x.ChildWorkflowExecutionCompletedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetChildWorkflowExecutionFailedEventAttributes() *ChildWorkflowExecutionFailedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ChildWorkflowExecutionFailedEventAttributes); ok { + return x.ChildWorkflowExecutionFailedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetChildWorkflowExecutionCanceledEventAttributes() *ChildWorkflowExecutionCanceledEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ChildWorkflowExecutionCanceledEventAttributes); ok { + return x.ChildWorkflowExecutionCanceledEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetChildWorkflowExecutionTimedOutEventAttributes() *ChildWorkflowExecutionTimedOutEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ChildWorkflowExecutionTimedOutEventAttributes); ok { + return x.ChildWorkflowExecutionTimedOutEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetChildWorkflowExecutionTerminatedEventAttributes() *ChildWorkflowExecutionTerminatedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ChildWorkflowExecutionTerminatedEventAttributes); ok { + return x.ChildWorkflowExecutionTerminatedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetSignalExternalWorkflowExecutionInitiatedEventAttributes() *SignalExternalWorkflowExecutionInitiatedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_SignalExternalWorkflowExecutionInitiatedEventAttributes); ok { + return x.SignalExternalWorkflowExecutionInitiatedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetSignalExternalWorkflowExecutionFailedEventAttributes() *SignalExternalWorkflowExecutionFailedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_SignalExternalWorkflowExecutionFailedEventAttributes); ok { + return x.SignalExternalWorkflowExecutionFailedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetExternalWorkflowExecutionSignaledEventAttributes() *ExternalWorkflowExecutionSignaledEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ExternalWorkflowExecutionSignaledEventAttributes); ok { + return x.ExternalWorkflowExecutionSignaledEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetUpsertWorkflowSearchAttributesEventAttributes() *UpsertWorkflowSearchAttributesEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_UpsertWorkflowSearchAttributesEventAttributes); ok { + return x.UpsertWorkflowSearchAttributesEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowExecutionUpdateAcceptedEventAttributes() *WorkflowExecutionUpdateAcceptedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowExecutionUpdateAcceptedEventAttributes); ok { + return x.WorkflowExecutionUpdateAcceptedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowExecutionUpdateRejectedEventAttributes() *WorkflowExecutionUpdateRejectedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowExecutionUpdateRejectedEventAttributes); ok { + return x.WorkflowExecutionUpdateRejectedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowExecutionUpdateCompletedEventAttributes() *WorkflowExecutionUpdateCompletedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowExecutionUpdateCompletedEventAttributes); ok { + return x.WorkflowExecutionUpdateCompletedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowPropertiesModifiedExternallyEventAttributes() *WorkflowPropertiesModifiedExternallyEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowPropertiesModifiedExternallyEventAttributes); ok { + return x.WorkflowPropertiesModifiedExternallyEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetActivityPropertiesModifiedExternallyEventAttributes() *ActivityPropertiesModifiedExternallyEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_ActivityPropertiesModifiedExternallyEventAttributes); ok { + return x.ActivityPropertiesModifiedExternallyEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowPropertiesModifiedEventAttributes() *WorkflowPropertiesModifiedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowPropertiesModifiedEventAttributes); ok { + return x.WorkflowPropertiesModifiedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetWorkflowExecutionUpdateAdmittedEventAttributes() *WorkflowExecutionUpdateAdmittedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_WorkflowExecutionUpdateAdmittedEventAttributes); ok { + return x.WorkflowExecutionUpdateAdmittedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetNexusOperationScheduledEventAttributes() *NexusOperationScheduledEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_NexusOperationScheduledEventAttributes); ok { + return x.NexusOperationScheduledEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetNexusOperationStartedEventAttributes() *NexusOperationStartedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_NexusOperationStartedEventAttributes); ok { + return x.NexusOperationStartedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetNexusOperationCompletedEventAttributes() *NexusOperationCompletedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_NexusOperationCompletedEventAttributes); ok { + return x.NexusOperationCompletedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetNexusOperationFailedEventAttributes() *NexusOperationFailedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_NexusOperationFailedEventAttributes); ok { + return x.NexusOperationFailedEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetNexusOperationCanceledEventAttributes() *NexusOperationCanceledEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_NexusOperationCanceledEventAttributes); ok { + return x.NexusOperationCanceledEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetNexusOperationTimedOutEventAttributes() *NexusOperationTimedOutEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_NexusOperationTimedOutEventAttributes); ok { + return x.NexusOperationTimedOutEventAttributes + } + return nil +} + +func (x *HistoryEvent) GetNexusOperationCancelRequestedEventAttributes() *NexusOperationCancelRequestedEventAttributes { + if x, ok := x.GetAttributes().(*HistoryEvent_NexusOperationCancelRequestedEventAttributes); ok { + return x.NexusOperationCancelRequestedEventAttributes + } + return nil +} + +type isHistoryEvent_Attributes interface { + isHistoryEvent_Attributes() +} + +type HistoryEvent_WorkflowExecutionStartedEventAttributes struct { + WorkflowExecutionStartedEventAttributes *WorkflowExecutionStartedEventAttributes `protobuf:"bytes,6,opt,name=workflow_execution_started_event_attributes,json=workflowExecutionStartedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowExecutionCompletedEventAttributes struct { + WorkflowExecutionCompletedEventAttributes *WorkflowExecutionCompletedEventAttributes `protobuf:"bytes,7,opt,name=workflow_execution_completed_event_attributes,json=workflowExecutionCompletedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowExecutionFailedEventAttributes struct { + WorkflowExecutionFailedEventAttributes *WorkflowExecutionFailedEventAttributes `protobuf:"bytes,8,opt,name=workflow_execution_failed_event_attributes,json=workflowExecutionFailedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowExecutionTimedOutEventAttributes struct { + WorkflowExecutionTimedOutEventAttributes *WorkflowExecutionTimedOutEventAttributes `protobuf:"bytes,9,opt,name=workflow_execution_timed_out_event_attributes,json=workflowExecutionTimedOutEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowTaskScheduledEventAttributes struct { + WorkflowTaskScheduledEventAttributes *WorkflowTaskScheduledEventAttributes `protobuf:"bytes,10,opt,name=workflow_task_scheduled_event_attributes,json=workflowTaskScheduledEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowTaskStartedEventAttributes struct { + WorkflowTaskStartedEventAttributes *WorkflowTaskStartedEventAttributes `protobuf:"bytes,11,opt,name=workflow_task_started_event_attributes,json=workflowTaskStartedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowTaskCompletedEventAttributes struct { + WorkflowTaskCompletedEventAttributes *WorkflowTaskCompletedEventAttributes `protobuf:"bytes,12,opt,name=workflow_task_completed_event_attributes,json=workflowTaskCompletedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowTaskTimedOutEventAttributes struct { + WorkflowTaskTimedOutEventAttributes *WorkflowTaskTimedOutEventAttributes `protobuf:"bytes,13,opt,name=workflow_task_timed_out_event_attributes,json=workflowTaskTimedOutEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowTaskFailedEventAttributes struct { + WorkflowTaskFailedEventAttributes *WorkflowTaskFailedEventAttributes `protobuf:"bytes,14,opt,name=workflow_task_failed_event_attributes,json=workflowTaskFailedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ActivityTaskScheduledEventAttributes struct { + ActivityTaskScheduledEventAttributes *ActivityTaskScheduledEventAttributes `protobuf:"bytes,15,opt,name=activity_task_scheduled_event_attributes,json=activityTaskScheduledEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ActivityTaskStartedEventAttributes struct { + ActivityTaskStartedEventAttributes *ActivityTaskStartedEventAttributes `protobuf:"bytes,16,opt,name=activity_task_started_event_attributes,json=activityTaskStartedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ActivityTaskCompletedEventAttributes struct { + ActivityTaskCompletedEventAttributes *ActivityTaskCompletedEventAttributes `protobuf:"bytes,17,opt,name=activity_task_completed_event_attributes,json=activityTaskCompletedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ActivityTaskFailedEventAttributes struct { + ActivityTaskFailedEventAttributes *ActivityTaskFailedEventAttributes `protobuf:"bytes,18,opt,name=activity_task_failed_event_attributes,json=activityTaskFailedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ActivityTaskTimedOutEventAttributes struct { + ActivityTaskTimedOutEventAttributes *ActivityTaskTimedOutEventAttributes `protobuf:"bytes,19,opt,name=activity_task_timed_out_event_attributes,json=activityTaskTimedOutEventAttributes,proto3,oneof"` +} + +type HistoryEvent_TimerStartedEventAttributes struct { + TimerStartedEventAttributes *TimerStartedEventAttributes `protobuf:"bytes,20,opt,name=timer_started_event_attributes,json=timerStartedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_TimerFiredEventAttributes struct { + TimerFiredEventAttributes *TimerFiredEventAttributes `protobuf:"bytes,21,opt,name=timer_fired_event_attributes,json=timerFiredEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ActivityTaskCancelRequestedEventAttributes struct { + ActivityTaskCancelRequestedEventAttributes *ActivityTaskCancelRequestedEventAttributes `protobuf:"bytes,22,opt,name=activity_task_cancel_requested_event_attributes,json=activityTaskCancelRequestedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ActivityTaskCanceledEventAttributes struct { + ActivityTaskCanceledEventAttributes *ActivityTaskCanceledEventAttributes `protobuf:"bytes,23,opt,name=activity_task_canceled_event_attributes,json=activityTaskCanceledEventAttributes,proto3,oneof"` +} + +type HistoryEvent_TimerCanceledEventAttributes struct { + TimerCanceledEventAttributes *TimerCanceledEventAttributes `protobuf:"bytes,24,opt,name=timer_canceled_event_attributes,json=timerCanceledEventAttributes,proto3,oneof"` +} + +type HistoryEvent_MarkerRecordedEventAttributes struct { + MarkerRecordedEventAttributes *MarkerRecordedEventAttributes `protobuf:"bytes,25,opt,name=marker_recorded_event_attributes,json=markerRecordedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowExecutionSignaledEventAttributes struct { + WorkflowExecutionSignaledEventAttributes *WorkflowExecutionSignaledEventAttributes `protobuf:"bytes,26,opt,name=workflow_execution_signaled_event_attributes,json=workflowExecutionSignaledEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowExecutionTerminatedEventAttributes struct { + WorkflowExecutionTerminatedEventAttributes *WorkflowExecutionTerminatedEventAttributes `protobuf:"bytes,27,opt,name=workflow_execution_terminated_event_attributes,json=workflowExecutionTerminatedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowExecutionCancelRequestedEventAttributes struct { + WorkflowExecutionCancelRequestedEventAttributes *WorkflowExecutionCancelRequestedEventAttributes `protobuf:"bytes,28,opt,name=workflow_execution_cancel_requested_event_attributes,json=workflowExecutionCancelRequestedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowExecutionCanceledEventAttributes struct { + WorkflowExecutionCanceledEventAttributes *WorkflowExecutionCanceledEventAttributes `protobuf:"bytes,29,opt,name=workflow_execution_canceled_event_attributes,json=workflowExecutionCanceledEventAttributes,proto3,oneof"` +} + +type HistoryEvent_RequestCancelExternalWorkflowExecutionInitiatedEventAttributes struct { + RequestCancelExternalWorkflowExecutionInitiatedEventAttributes *RequestCancelExternalWorkflowExecutionInitiatedEventAttributes `protobuf:"bytes,30,opt,name=request_cancel_external_workflow_execution_initiated_event_attributes,json=requestCancelExternalWorkflowExecutionInitiatedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_RequestCancelExternalWorkflowExecutionFailedEventAttributes struct { + RequestCancelExternalWorkflowExecutionFailedEventAttributes *RequestCancelExternalWorkflowExecutionFailedEventAttributes `protobuf:"bytes,31,opt,name=request_cancel_external_workflow_execution_failed_event_attributes,json=requestCancelExternalWorkflowExecutionFailedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ExternalWorkflowExecutionCancelRequestedEventAttributes struct { + ExternalWorkflowExecutionCancelRequestedEventAttributes *ExternalWorkflowExecutionCancelRequestedEventAttributes `protobuf:"bytes,32,opt,name=external_workflow_execution_cancel_requested_event_attributes,json=externalWorkflowExecutionCancelRequestedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowExecutionContinuedAsNewEventAttributes struct { + WorkflowExecutionContinuedAsNewEventAttributes *WorkflowExecutionContinuedAsNewEventAttributes `protobuf:"bytes,33,opt,name=workflow_execution_continued_as_new_event_attributes,json=workflowExecutionContinuedAsNewEventAttributes,proto3,oneof"` +} + +type HistoryEvent_StartChildWorkflowExecutionInitiatedEventAttributes struct { + StartChildWorkflowExecutionInitiatedEventAttributes *StartChildWorkflowExecutionInitiatedEventAttributes `protobuf:"bytes,34,opt,name=start_child_workflow_execution_initiated_event_attributes,json=startChildWorkflowExecutionInitiatedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_StartChildWorkflowExecutionFailedEventAttributes struct { + StartChildWorkflowExecutionFailedEventAttributes *StartChildWorkflowExecutionFailedEventAttributes `protobuf:"bytes,35,opt,name=start_child_workflow_execution_failed_event_attributes,json=startChildWorkflowExecutionFailedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ChildWorkflowExecutionStartedEventAttributes struct { + ChildWorkflowExecutionStartedEventAttributes *ChildWorkflowExecutionStartedEventAttributes `protobuf:"bytes,36,opt,name=child_workflow_execution_started_event_attributes,json=childWorkflowExecutionStartedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ChildWorkflowExecutionCompletedEventAttributes struct { + ChildWorkflowExecutionCompletedEventAttributes *ChildWorkflowExecutionCompletedEventAttributes `protobuf:"bytes,37,opt,name=child_workflow_execution_completed_event_attributes,json=childWorkflowExecutionCompletedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ChildWorkflowExecutionFailedEventAttributes struct { + ChildWorkflowExecutionFailedEventAttributes *ChildWorkflowExecutionFailedEventAttributes `protobuf:"bytes,38,opt,name=child_workflow_execution_failed_event_attributes,json=childWorkflowExecutionFailedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ChildWorkflowExecutionCanceledEventAttributes struct { + ChildWorkflowExecutionCanceledEventAttributes *ChildWorkflowExecutionCanceledEventAttributes `protobuf:"bytes,39,opt,name=child_workflow_execution_canceled_event_attributes,json=childWorkflowExecutionCanceledEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ChildWorkflowExecutionTimedOutEventAttributes struct { + ChildWorkflowExecutionTimedOutEventAttributes *ChildWorkflowExecutionTimedOutEventAttributes `protobuf:"bytes,40,opt,name=child_workflow_execution_timed_out_event_attributes,json=childWorkflowExecutionTimedOutEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ChildWorkflowExecutionTerminatedEventAttributes struct { + ChildWorkflowExecutionTerminatedEventAttributes *ChildWorkflowExecutionTerminatedEventAttributes `protobuf:"bytes,41,opt,name=child_workflow_execution_terminated_event_attributes,json=childWorkflowExecutionTerminatedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_SignalExternalWorkflowExecutionInitiatedEventAttributes struct { + SignalExternalWorkflowExecutionInitiatedEventAttributes *SignalExternalWorkflowExecutionInitiatedEventAttributes `protobuf:"bytes,42,opt,name=signal_external_workflow_execution_initiated_event_attributes,json=signalExternalWorkflowExecutionInitiatedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_SignalExternalWorkflowExecutionFailedEventAttributes struct { + SignalExternalWorkflowExecutionFailedEventAttributes *SignalExternalWorkflowExecutionFailedEventAttributes `protobuf:"bytes,43,opt,name=signal_external_workflow_execution_failed_event_attributes,json=signalExternalWorkflowExecutionFailedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ExternalWorkflowExecutionSignaledEventAttributes struct { + ExternalWorkflowExecutionSignaledEventAttributes *ExternalWorkflowExecutionSignaledEventAttributes `protobuf:"bytes,44,opt,name=external_workflow_execution_signaled_event_attributes,json=externalWorkflowExecutionSignaledEventAttributes,proto3,oneof"` +} + +type HistoryEvent_UpsertWorkflowSearchAttributesEventAttributes struct { + UpsertWorkflowSearchAttributesEventAttributes *UpsertWorkflowSearchAttributesEventAttributes `protobuf:"bytes,45,opt,name=upsert_workflow_search_attributes_event_attributes,json=upsertWorkflowSearchAttributesEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowExecutionUpdateAcceptedEventAttributes struct { + WorkflowExecutionUpdateAcceptedEventAttributes *WorkflowExecutionUpdateAcceptedEventAttributes `protobuf:"bytes,46,opt,name=workflow_execution_update_accepted_event_attributes,json=workflowExecutionUpdateAcceptedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowExecutionUpdateRejectedEventAttributes struct { + WorkflowExecutionUpdateRejectedEventAttributes *WorkflowExecutionUpdateRejectedEventAttributes `protobuf:"bytes,47,opt,name=workflow_execution_update_rejected_event_attributes,json=workflowExecutionUpdateRejectedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowExecutionUpdateCompletedEventAttributes struct { + WorkflowExecutionUpdateCompletedEventAttributes *WorkflowExecutionUpdateCompletedEventAttributes `protobuf:"bytes,48,opt,name=workflow_execution_update_completed_event_attributes,json=workflowExecutionUpdateCompletedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowPropertiesModifiedExternallyEventAttributes struct { + WorkflowPropertiesModifiedExternallyEventAttributes *WorkflowPropertiesModifiedExternallyEventAttributes `protobuf:"bytes,49,opt,name=workflow_properties_modified_externally_event_attributes,json=workflowPropertiesModifiedExternallyEventAttributes,proto3,oneof"` +} + +type HistoryEvent_ActivityPropertiesModifiedExternallyEventAttributes struct { + ActivityPropertiesModifiedExternallyEventAttributes *ActivityPropertiesModifiedExternallyEventAttributes `protobuf:"bytes,50,opt,name=activity_properties_modified_externally_event_attributes,json=activityPropertiesModifiedExternallyEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowPropertiesModifiedEventAttributes struct { + WorkflowPropertiesModifiedEventAttributes *WorkflowPropertiesModifiedEventAttributes `protobuf:"bytes,51,opt,name=workflow_properties_modified_event_attributes,json=workflowPropertiesModifiedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_WorkflowExecutionUpdateAdmittedEventAttributes struct { + WorkflowExecutionUpdateAdmittedEventAttributes *WorkflowExecutionUpdateAdmittedEventAttributes `protobuf:"bytes,52,opt,name=workflow_execution_update_admitted_event_attributes,json=workflowExecutionUpdateAdmittedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_NexusOperationScheduledEventAttributes struct { + NexusOperationScheduledEventAttributes *NexusOperationScheduledEventAttributes `protobuf:"bytes,53,opt,name=nexus_operation_scheduled_event_attributes,json=nexusOperationScheduledEventAttributes,proto3,oneof"` +} + +type HistoryEvent_NexusOperationStartedEventAttributes struct { + NexusOperationStartedEventAttributes *NexusOperationStartedEventAttributes `protobuf:"bytes,54,opt,name=nexus_operation_started_event_attributes,json=nexusOperationStartedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_NexusOperationCompletedEventAttributes struct { + NexusOperationCompletedEventAttributes *NexusOperationCompletedEventAttributes `protobuf:"bytes,55,opt,name=nexus_operation_completed_event_attributes,json=nexusOperationCompletedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_NexusOperationFailedEventAttributes struct { + NexusOperationFailedEventAttributes *NexusOperationFailedEventAttributes `protobuf:"bytes,56,opt,name=nexus_operation_failed_event_attributes,json=nexusOperationFailedEventAttributes,proto3,oneof"` +} + +type HistoryEvent_NexusOperationCanceledEventAttributes struct { + NexusOperationCanceledEventAttributes *NexusOperationCanceledEventAttributes `protobuf:"bytes,57,opt,name=nexus_operation_canceled_event_attributes,json=nexusOperationCanceledEventAttributes,proto3,oneof"` +} + +type HistoryEvent_NexusOperationTimedOutEventAttributes struct { + NexusOperationTimedOutEventAttributes *NexusOperationTimedOutEventAttributes `protobuf:"bytes,58,opt,name=nexus_operation_timed_out_event_attributes,json=nexusOperationTimedOutEventAttributes,proto3,oneof"` +} + +type HistoryEvent_NexusOperationCancelRequestedEventAttributes struct { + NexusOperationCancelRequestedEventAttributes *NexusOperationCancelRequestedEventAttributes `protobuf:"bytes,59,opt,name=nexus_operation_cancel_requested_event_attributes,json=nexusOperationCancelRequestedEventAttributes,proto3,oneof"` +} + +func (*HistoryEvent_WorkflowExecutionStartedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowExecutionCompletedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowExecutionFailedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowExecutionTimedOutEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowTaskScheduledEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowTaskStartedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowTaskCompletedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowTaskTimedOutEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowTaskFailedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_ActivityTaskScheduledEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_ActivityTaskStartedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_ActivityTaskCompletedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_ActivityTaskFailedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_ActivityTaskTimedOutEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_TimerStartedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_TimerFiredEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_ActivityTaskCancelRequestedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_ActivityTaskCanceledEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_TimerCanceledEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_MarkerRecordedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowExecutionSignaledEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowExecutionTerminatedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowExecutionCancelRequestedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowExecutionCanceledEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_RequestCancelExternalWorkflowExecutionInitiatedEventAttributes) isHistoryEvent_Attributes() { +} + +func (*HistoryEvent_RequestCancelExternalWorkflowExecutionFailedEventAttributes) isHistoryEvent_Attributes() { +} + +func (*HistoryEvent_ExternalWorkflowExecutionCancelRequestedEventAttributes) isHistoryEvent_Attributes() { +} + +func (*HistoryEvent_WorkflowExecutionContinuedAsNewEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_StartChildWorkflowExecutionInitiatedEventAttributes) isHistoryEvent_Attributes() { +} + +func (*HistoryEvent_StartChildWorkflowExecutionFailedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_ChildWorkflowExecutionStartedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_ChildWorkflowExecutionCompletedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_ChildWorkflowExecutionFailedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_ChildWorkflowExecutionCanceledEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_ChildWorkflowExecutionTimedOutEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_ChildWorkflowExecutionTerminatedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_SignalExternalWorkflowExecutionInitiatedEventAttributes) isHistoryEvent_Attributes() { +} + +func (*HistoryEvent_SignalExternalWorkflowExecutionFailedEventAttributes) isHistoryEvent_Attributes() { +} + +func (*HistoryEvent_ExternalWorkflowExecutionSignaledEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_UpsertWorkflowSearchAttributesEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowExecutionUpdateAcceptedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowExecutionUpdateRejectedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowExecutionUpdateCompletedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowPropertiesModifiedExternallyEventAttributes) isHistoryEvent_Attributes() { +} + +func (*HistoryEvent_ActivityPropertiesModifiedExternallyEventAttributes) isHistoryEvent_Attributes() { +} + +func (*HistoryEvent_WorkflowPropertiesModifiedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_WorkflowExecutionUpdateAdmittedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_NexusOperationScheduledEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_NexusOperationStartedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_NexusOperationCompletedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_NexusOperationFailedEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_NexusOperationCanceledEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_NexusOperationTimedOutEventAttributes) isHistoryEvent_Attributes() {} + +func (*HistoryEvent_NexusOperationCancelRequestedEventAttributes) isHistoryEvent_Attributes() {} + +type History struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Events []*HistoryEvent `protobuf:"bytes,1,rep,name=events,proto3" json:"events,omitempty"` +} + +func (x *History) Reset() { + *x = History{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[55] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *History) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*History) ProtoMessage() {} + +func (x *History) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_history_v1_message_proto_msgTypes[55] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use History.ProtoReflect.Descriptor instead. +func (*History) Descriptor() ([]byte, []int) { + return file_temporal_api_history_v1_message_proto_rawDescGZIP(), []int{55} +} + +func (x *History) GetEvents() []*HistoryEvent { + if x != nil { + return x.Events + } + return nil +} + +var File_temporal_api_history_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_history_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x25, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x68, 0x69, + 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x17, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x1a, 0x1e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x75, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x26, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, + 0x2f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x1a, 0x28, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x63, 0x61, + 0x75, 0x73, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x22, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, + 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, + 0x31, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2f, + 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x1a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x75, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x26, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x30, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x73, 0x64, 0x6b, 0x2f, 0x76, 0x31, 0x2f, 0x74, + 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x6d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x73, 0x64, 0x6b, 0x2f, 0x76, 0x31, 0x2f, 0x75, + 0x73, 0x65, 0x72, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x22, 0xd3, 0x12, 0x0a, 0x27, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x4d, 0x0a, 0x0d, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, + 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x19, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, + 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x43, 0x0a, 0x1c, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, + 0x69, 0x64, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x69, 0x0a, 0x19, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x17, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x19, 0x70, + 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x16, 0x70, 0x61, + 0x72, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, + 0x75, 0x65, 0x75, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, + 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x09, + 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, + 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x05, 0x69, 0x6e, 0x70, + 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5b, 0x0a, 0x1a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x18, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, + 0x0a, 0x14, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, + 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x51, 0x0a, 0x15, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, + 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, 0x0a, 0x1a, 0x63, 0x6f, + 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x64, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x63, + 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, 0x09, 0x69, 0x6e, 0x69, 0x74, + 0x69, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2d, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x41, 0x73, 0x4e, 0x65, 0x77, 0x49, + 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x09, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, + 0x6f, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x51, 0x0a, 0x11, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, + 0x65, 0x64, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x52, 0x10, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5a, 0x0a, 0x16, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6d, + 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x0d, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x73, 0x52, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x19, + 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x61, 0x6c, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x16, + 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x61, 0x6c, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x74, 0x79, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x37, 0x0a, 0x16, 0x66, 0x69, 0x72, 0x73, 0x74, + 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, + 0x18, 0x10, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x66, 0x69, 0x72, 0x73, 0x74, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, + 0x0c, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x11, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, + 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, + 0x18, 0x12, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x6b, 0x0a, 0x22, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x1f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, 0x0a, 0x0d, 0x63, 0x72, + 0x6f, 0x6e, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0c, 0x63, 0x72, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x1b, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x18, + 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x18, + 0x66, 0x69, 0x72, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, + 0x6b, 0x42, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, 0x04, 0x6d, + 0x65, 0x6d, 0x6f, 0x18, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x4d, 0x65, 0x6d, 0x6f, 0x52, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, + 0x0a, 0x11, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x52, 0x10, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5e, 0x0a, 0x16, 0x70, 0x72, 0x65, 0x76, 0x5f, 0x61, 0x75, + 0x74, 0x6f, 0x5f, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x18, 0x18, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, + 0x73, 0x65, 0x74, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x52, 0x13, 0x70, 0x72, 0x65, 0x76, 0x41, + 0x75, 0x74, 0x6f, 0x52, 0x65, 0x73, 0x65, 0x74, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, + 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x1e, 0x70, 0x61, + 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x1a, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x1b, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x1c, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x60, 0x0a, 0x14, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x1d, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x12, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x6d, 0x70, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x57, 0x0a, 0x14, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x61, + 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x73, 0x18, 0x1e, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x52, 0x13, 0x63, + 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x65, 0x0a, 0x17, 0x72, 0x6f, 0x6f, 0x74, 0x5f, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x1f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x15, 0x72, + 0x6f, 0x6f, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x69, 0x6e, 0x68, 0x65, 0x72, + 0x69, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x20, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x10, 0x69, 0x6e, 0x68, 0x65, 0x72, 0x69, 0x74, 0x65, 0x64, 0x42, 0x75, 0x69, 0x6c, + 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xea, 0x01, 0x0a, 0x29, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x73, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, + 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, + 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, + 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x33, 0x0a, 0x14, 0x6e, 0x65, 0x77, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x11, 0x6e, 0x65, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xb1, 0x02, 0x0a, 0x26, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, + 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x12, 0x3e, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, + 0x75, 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x46, 0x0a, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x33, 0x0a, 0x14, + 0x6e, 0x65, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x75, 0x6e, + 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x6e, 0x65, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xa7, + 0x01, 0x0a, 0x28, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x64, 0x4f, 0x75, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x46, 0x0a, 0x0b, 0x72, 0x65, 0x74, + 0x72, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, + 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x33, + 0x0a, 0x14, 0x6e, 0x65, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, + 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x6e, 0x65, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x22, 0xe6, 0x08, 0x0a, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x64, 0x41, 0x73, + 0x4e, 0x65, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x12, 0x33, 0x0a, 0x14, 0x6e, 0x65, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, + 0x6e, 0x65, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x75, 0x6e, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x47, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, + 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x73, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, + 0x0a, 0x14, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, + 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x51, 0x0a, 0x15, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x13, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x54, + 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, + 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, + 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x53, 0x0a, 0x16, 0x62, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x5f, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x08, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x14, 0x62, 0x61, 0x63, 0x6b, + 0x6f, 0x66, 0x66, 0x53, 0x74, 0x61, 0x72, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, 0x09, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x6f, 0x72, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, + 0x74, 0x69, 0x6e, 0x75, 0x65, 0x41, 0x73, 0x4e, 0x65, 0x77, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, + 0x6f, 0x72, 0x52, 0x09, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x6f, 0x72, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5a, + 0x0a, 0x16, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x14, 0x6c, 0x61, + 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, + 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, + 0x0a, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x6d, 0x6f, 0x52, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x59, 0x0a, 0x11, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x52, 0x10, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x69, 0x6e, 0x68, 0x65, + 0x72, 0x69, 0x74, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x0f, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x68, 0x65, 0x72, 0x69, 0x74, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xe1, 0x01, 0x0a, 0x24, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x47, 0x0a, 0x0a, + 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, + 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, + 0x65, 0x75, 0x65, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x52, 0x0a, 0x16, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6c, + 0x6f, 0x73, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x54, 0x6f, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0x9c, + 0x03, 0x0a, 0x22, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, + 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, + 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x39, 0x0a, 0x17, 0x73, 0x75, 0x67, 0x67, 0x65, 0x73, 0x74, 0x5f, 0x63, 0x6f, + 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x5f, 0x61, 0x73, 0x5f, 0x6e, 0x65, 0x77, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x14, 0x73, 0x75, 0x67, 0x67, 0x65, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x69, 0x6e, + 0x75, 0x65, 0x41, 0x73, 0x4e, 0x65, 0x77, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x68, + 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x53, 0x69, + 0x7a, 0x65, 0x42, 0x79, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x55, 0x0a, 0x0e, 0x77, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, + 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0d, 0x77, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, + 0x19, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x5f, 0x72, 0x65, 0x64, 0x69, 0x72, 0x65, + 0x63, 0x74, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x16, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x43, + 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x22, 0xe0, 0x03, 0x0a, 0x24, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, + 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2b, 0x0a, 0x0f, 0x62, 0x69, + 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0e, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, + 0x75, 0x6d, 0x42, 0x02, 0x68, 0x00, 0x12, 0x55, 0x0a, 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, 0x0c, 0x73, 0x64, 0x6b, 0x5f, + 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, + 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, + 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, + 0x0b, 0x73, 0x64, 0x6b, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x59, 0x0a, 0x11, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x4d, 0x65, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x52, 0x10, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd0, 0x01, 0x0a, 0x23, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x64, 0x4f, 0x75, 0x74, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x49, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x22, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, + 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, + 0x0b, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, + 0xa9, 0x04, 0x0a, 0x21, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x46, + 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, + 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x48, 0x0a, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, + 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, + 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x61, 0x75, 0x73, 0x65, 0x52, 0x05, 0x63, 0x61, 0x75, + 0x73, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x22, 0x0a, 0x0b, 0x62, 0x61, 0x73, 0x65, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x62, 0x61, 0x73, 0x65, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x0a, 0x6e, 0x65, 0x77, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x69, + 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6e, 0x65, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, + 0x66, 0x6f, 0x72, 0x6b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x2b, 0x0a, 0x0f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x63, 0x68, + 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x62, 0x69, 0x6e, + 0x61, 0x72, 0x79, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x42, 0x02, 0x68, 0x00, 0x12, 0x55, + 0x0a, 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x6d, 0x70, 0x52, + 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x84, 0x07, 0x0a, 0x24, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, + 0x6b, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0b, 0x61, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x61, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x61, + 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, + 0x74, 0x79, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, + 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, + 0x75, 0x65, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, + 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x09, + 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x06, + 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x73, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x58, 0x0a, 0x19, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6c, + 0x6f, 0x73, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x16, 0x73, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x54, 0x6f, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, + 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x58, 0x0a, 0x19, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x5f, 0x74, 0x6f, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, + 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x16, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x6f, 0x53, 0x74, 0x61, 0x72, 0x74, + 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x52, 0x0a, 0x16, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x6f, 0x43, 0x6c, 0x6f, 0x73, 0x65, + 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x11, 0x68, 0x65, + 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x0a, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x10, + 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, + 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x0c, 0x72, + 0x65, 0x74, 0x72, 0x79, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x23, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x35, 0x0a, 0x15, 0x75, 0x73, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x0d, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x12, 0x75, 0x73, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x75, 0x69, + 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x4a, 0x04, 0x08, 0x03, 0x10, 0x04, 0x22, 0x96, + 0x03, 0x0a, 0x22, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, + 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x47, 0x0a, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, + 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x0b, 0x6c, 0x61, 0x73, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x55, 0x0a, 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x53, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x19, 0x62, 0x75, 0x69, 0x6c, 0x64, + 0x5f, 0x69, 0x64, 0x5f, 0x72, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x5f, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x16, 0x62, 0x75, 0x69, 0x6c, 0x64, + 0x49, 0x64, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, + 0x42, 0x02, 0x68, 0x00, 0x22, 0xbb, 0x02, 0x0a, 0x24, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, + 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x06, 0x72, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, + 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x55, 0x0a, 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x82, 0x03, 0x0a, 0x21, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, + 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x3e, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, + 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x46, + 0x0a, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x55, 0x0a, 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8d, 0x02, 0x0a, 0x23, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, + 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x64, 0x4f, 0x75, 0x74, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x3e, 0x0a, 0x07, 0x66, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, + 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x46, 0x0a, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, + 0x52, 0x65, 0x74, 0x72, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xaa, 0x01, 0x0a, 0x2a, 0x41, 0x63, 0x74, + 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, + 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x88, + 0x03, 0x0a, 0x23, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x61, + 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x12, 0x3e, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, + 0x6c, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x6c, 0x61, 0x74, 0x65, 0x73, + 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x55, 0x0a, 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xda, 0x01, 0x0a, 0x1b, 0x54, 0x69, 0x6d, 0x65, 0x72, + 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x1d, 0x0a, 0x08, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x50, 0x0a, 0x15, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x6f, 0x5f, 0x66, + 0x69, 0x72, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x54, 0x6f, 0x46, 0x69, 0x72, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, + 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x68, 0x0a, 0x19, 0x54, 0x69, 0x6d, + 0x65, 0x72, 0x46, 0x69, 0x72, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x1d, 0x0a, 0x08, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd7, + 0x01, 0x0a, 0x1c, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x1d, 0x0a, + 0x08, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x73, + 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x1c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x9d, + 0x02, 0x0a, 0x2f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, + 0x18, 0x0a, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x63, 0x61, 0x75, 0x73, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x1b, 0x65, 0x78, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x18, 0x65, 0x78, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x6d, 0x0a, 0x1b, 0x65, 0x78, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x19, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, + 0x22, 0xb6, 0x01, 0x0a, 0x28, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x4a, 0x0a, 0x20, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, + 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x73, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x22, 0xd5, 0x03, 0x0a, 0x1d, 0x4d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x63, 0x6f, 0x72, + 0x64, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x12, 0x23, 0x0a, 0x0b, 0x6d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x4e, 0x61, 0x6d, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x61, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x61, + 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x2e, 0x44, 0x65, 0x74, 0x61, 0x69, + 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, + 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x68, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, + 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x1a, 0x64, 0x0a, 0x0c, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x99, 0x03, 0x0a, 0x28, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x23, 0x0a, + 0x0b, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, + 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x41, 0x0a, 0x1b, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, + 0x74, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x73, 0x6b, 0x69, 0x70, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, + 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x6d, 0x0a, 0x1b, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x19, + 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xa8, 0x01, 0x0a, 0x2a, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x1a, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, + 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x73, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0xa1, + 0x03, 0x0a, 0x3e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x45, + 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x4a, + 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, + 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x32, 0x0a, 0x13, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, + 0x63, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x4f, 0x6e, 0x6c, 0x79, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x1a, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, + 0xdd, 0x03, 0x0a, 0x3b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, + 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x5b, 0x0a, 0x05, 0x63, + 0x61, 0x75, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x41, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, + 0x31, 0x2e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, + 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x61, 0x75, 0x73, 0x65, 0x52, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, + 0x0c, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, + 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x63, + 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, + 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x22, 0x92, 0x02, 0x0a, 0x37, 0x45, 0x78, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x12, 0x30, 0x0a, 0x12, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x69, 0x6e, + 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, + 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, + 0x22, 0x9b, 0x04, 0x0a, 0x37, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, + 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x1c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, + 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, + 0x69, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x73, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, + 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x32, 0x0a, 0x13, 0x63, 0x68, + 0x69, 0x6c, 0x64, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x4f, 0x6e, 0x6c, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x68, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd6, 0x03, 0x0a, 0x34, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x45, + 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x5b, 0x0a, 0x05, 0x63, + 0x61, 0x75, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x41, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, + 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, + 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x61, 0x75, 0x73, 0x65, 0x52, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x69, + 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x63, + 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6e, + 0x74, 0x72, 0x6f, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x22, 0xa9, 0x02, 0x0a, 0x30, 0x45, 0x78, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, 0x69, 0x6e, + 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x63, 0x6f, 0x6e, + 0x74, 0x72, 0x6f, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, + 0x6f, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd6, 0x01, 0x0a, 0x2d, 0x55, 0x70, 0x73, 0x65, 0x72, 0x74, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, + 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, + 0x0a, 0x11, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x52, 0x10, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0xbe, 0x01, 0x0a, 0x29, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, + 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x45, 0x0a, 0x0d, 0x75, 0x70, 0x73, 0x65, + 0x72, 0x74, 0x65, 0x64, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x6d, 0x6f, 0x52, 0x0c, 0x75, 0x70, 0x73, + 0x65, 0x72, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x6d, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x22, 0x9b, 0x0a, 0x0a, + 0x33, 0x53, 0x74, 0x61, 0x72, 0x74, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x69, 0x74, 0x69, + 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, + 0x69, 0x64, 0x18, 0x12, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, + 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, + 0x75, 0x65, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x09, 0x74, + 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x69, + 0x6e, 0x70, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, + 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5b, 0x0a, 0x1a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, + 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x18, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, + 0x14, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, + 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x51, 0x0a, 0x15, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x13, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x54, + 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x13, 0x70, 0x61, 0x72, + 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x72, + 0x65, 0x6e, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x11, 0x70, + 0x61, 0x72, 0x65, 0x6e, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x18, 0x0a, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, + 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x69, 0x0a, 0x18, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x5f, 0x72, 0x65, 0x75, 0x73, 0x65, 0x5f, 0x70, 0x6f, 0x6c, + 0x69, 0x63, 0x79, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x52, 0x65, 0x75, 0x73, 0x65, 0x50, + 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x15, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, + 0x52, 0x65, 0x75, 0x73, 0x65, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, + 0x0a, 0x0c, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x0d, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, + 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, + 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, 0x0a, 0x0d, 0x63, 0x72, 0x6f, 0x6e, 0x5f, 0x73, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, + 0x72, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, + 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x18, + 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x6d, + 0x6f, 0x52, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, 0x11, 0x73, 0x65, + 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, + 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x61, + 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x10, 0x73, 0x65, + 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x69, 0x6e, 0x68, 0x65, 0x72, 0x69, 0x74, 0x5f, 0x62, 0x75, 0x69, + 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x13, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x68, 0x65, + 0x72, 0x69, 0x74, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xe4, 0x03, + 0x0a, 0x30, 0x53, 0x74, 0x61, 0x72, 0x74, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x25, 0x0a, 0x0c, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, + 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x57, 0x0a, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x3d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x43, + 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x61, 0x75, 0x73, 0x65, 0x52, 0x05, + 0x63, 0x61, 0x75, 0x73, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, + 0x72, 0x6f, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, + 0x6f, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, + 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x10, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, + 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x92, 0x03, + 0x0a, 0x2c, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x20, 0x0a, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x69, 0x6e, + 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, + 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x22, + 0xc4, 0x03, 0x0a, 0x2e, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x12, 0x3c, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, + 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, + 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, + 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x10, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, + 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x8b, 0x04, 0x0a, 0x2b, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, + 0x3e, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, + 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, + 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, + 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, + 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x10, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, + 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x46, 0x0a, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, + 0x74, 0x72, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xc5, 0x03, 0x0a, 0x2d, 0x43, 0x68, 0x69, 0x6c, 0x64, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x3e, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, + 0x0a, 0x12, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, + 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, + 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xcd, 0x03, 0x0a, 0x2d, 0x43, 0x68, + 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x64, 0x4f, 0x75, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, + 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, + 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, + 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, + 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x46, 0x0a, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x22, 0x87, 0x03, 0x0a, 0x2f, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x65, 0x72, + 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x30, 0x0a, 0x12, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x69, 0x6e, 0x69, 0x74, + 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xbc, 0x03, 0x0a, 0x33, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, + 0x73, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x6c, 0x79, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x12, 0x28, 0x0a, 0x0e, 0x6e, 0x65, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, + 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x6e, 0x65, 0x77, 0x54, 0x61, 0x73, 0x6b, + 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x58, 0x0a, 0x19, 0x6e, 0x65, 0x77, 0x5f, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x6f, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x16, 0x6e, 0x65, 0x77, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x54, 0x61, 0x73, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x56, + 0x0a, 0x18, 0x6e, 0x65, 0x77, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x72, 0x75, + 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x15, 0x6e, 0x65, 0x77, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x62, 0x0a, 0x1e, 0x6e, 0x65, 0x77, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x1b, 0x6e, 0x65, 0x77, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x45, 0x0a, 0x0d, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x6d, 0x65, + 0x6d, 0x6f, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x4d, 0x65, 0x6d, 0x6f, 0x52, 0x0c, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x6d, + 0x6f, 0x42, 0x02, 0x68, 0x00, 0x22, 0xba, 0x01, 0x0a, 0x33, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, + 0x74, 0x79, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x4d, 0x6f, 0x64, 0x69, 0x66, + 0x69, 0x65, 0x64, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x51, 0x0a, 0x10, 0x6e, 0x65, + 0x77, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, + 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0e, 0x6e, 0x65, 0x77, 0x52, 0x65, 0x74, 0x72, 0x79, + 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0xcd, 0x02, 0x0a, 0x2e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x41, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x34, 0x0a, 0x14, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x5f, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, + 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, + 0x1b, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x18, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x52, 0x0a, + 0x24, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x5f, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x69, 0x6e, 0x67, 0x5f, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x20, 0x61, 0x63, 0x63, 0x65, 0x70, + 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, + 0x69, 0x6e, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4e, 0x0a, + 0x10, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x0f, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd6, 0x01, 0x0a, 0x2f, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x34, 0x0a, 0x04, 0x6d, + 0x65, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x04, 0x6d, 0x65, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x2e, 0x0a, 0x11, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x07, + 0x6f, 0x75, 0x74, 0x63, 0x6f, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x75, 0x74, 0x63, 0x6f, 0x6d, 0x65, 0x52, 0x07, 0x6f, 0x75, + 0x74, 0x63, 0x6f, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8d, 0x03, 0x0a, 0x2e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x34, 0x0a, 0x14, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x5f, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x49, + 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x1b, + 0x72, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x18, 0x72, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x52, 0x0a, 0x24, + 0x72, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, + 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x69, 0x6e, 0x67, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x20, 0x72, 0x65, 0x6a, 0x65, 0x63, 0x74, + 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x69, + 0x6e, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4e, 0x0a, 0x10, + 0x72, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x0f, 0x72, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, + 0x75, 0x72, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xbd, 0x01, 0x0a, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x41, 0x64, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x3d, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x4c, 0x0a, 0x06, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x30, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x41, + 0x64, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4f, 0x72, 0x69, 0x67, 0x69, + 0x6e, 0x52, 0x06, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xf2, 0x04, + 0x0a, 0x26, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x1e, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x39, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, + 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x58, 0x0a, 0x19, 0x73, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x16, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x6f, + 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x77, 0x0a, 0x0c, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x06, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x50, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x65, + 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x2e, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x0b, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, + 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, + 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x1a, 0x46, 0x0a, 0x10, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x3a, 0x02, 0x38, 0x01, 0x22, 0xa2, 0x01, 0x0a, 0x24, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, 0x73, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x22, 0xba, 0x01, 0x0a, 0x26, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3b, 0x0a, 0x06, 0x72, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xba, 0x01, 0x0a, 0x23, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x66, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, + 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xbc, 0x01, 0x0a, 0x25, 0x4e, + 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, + 0x64, 0x4f, 0x75, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, + 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x22, 0xbc, 0x01, 0x0a, 0x25, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xac, 0x01, 0x0a, 0x2c, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x10, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1c, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xae, 0x4d, + 0x0a, 0x0c, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x1d, 0x0a, + 0x08, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x0a, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, + 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x43, 0x0a, 0x0a, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, + 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1b, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x2f, 0x0a, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x79, 0x5f, 0x69, + 0x67, 0x6e, 0x6f, 0x72, 0x65, 0x18, 0xac, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x77, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x4d, 0x61, 0x79, 0x49, 0x67, 0x6e, 0x6f, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x4b, 0x0a, 0x0d, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, + 0xad, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, 0x65, 0x72, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x0c, 0x75, 0x73, 0x65, 0x72, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, 0x37, 0x0a, 0x05, 0x6c, 0x69, 0x6e, 0x6b, + 0x73, 0x18, 0xae, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x4c, 0x69, 0x6e, 0x6b, 0x52, 0x05, 0x6c, 0x69, 0x6e, 0x6b, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0xa4, 0x01, 0x0a, 0x2b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, + 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x48, 0x00, 0x52, 0x27, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xaa, + 0x01, 0x0a, 0x2d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, + 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x29, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0xa1, 0x01, 0x0a, 0x2a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, + 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x26, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0xa8, 0x01, 0x0a, 0x2d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x64, 0x5f, 0x6f, 0x75, 0x74, 0x5f, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, + 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, + 0x69, 0x6d, 0x65, 0x64, 0x4f, 0x75, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x28, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x64, 0x4f, 0x75, 0x74, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x9b, 0x01, 0x0a, 0x28, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x0a, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, + 0x00, 0x52, 0x24, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x95, 0x01, 0x0a, 0x26, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, + 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, + 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x48, 0x00, 0x52, 0x22, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, + 0x6b, 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x9b, 0x01, 0x0a, 0x28, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, + 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, + 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x24, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x99, 0x01, 0x0a, 0x28, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, + 0x73, 0x6b, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x64, 0x5f, 0x6f, 0x75, 0x74, 0x5f, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x3c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x64, 0x4f, 0x75, 0x74, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x23, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x64, + 0x4f, 0x75, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x92, 0x01, 0x0a, 0x25, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x0e, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x21, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x9b, 0x01, 0x0a, 0x28, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, + 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x0f, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x48, 0x00, 0x52, 0x24, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x53, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x95, 0x01, 0x0a, 0x26, 0x61, + 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, + 0x31, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, + 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x22, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, + 0x73, 0x6b, 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x9b, 0x01, 0x0a, 0x28, + 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, + 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, + 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, + 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x24, 0x61, 0x63, 0x74, 0x69, 0x76, + 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x92, 0x01, 0x0a, 0x25, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x74, + 0x61, 0x73, 0x6b, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, + 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x3a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, + 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, + 0x79, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x21, 0x61, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x99, 0x01, 0x0a, 0x28, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x61, 0x73, + 0x6b, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x64, 0x5f, 0x6f, 0x75, 0x74, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x3c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, + 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, + 0x79, 0x54, 0x61, 0x73, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x64, 0x4f, 0x75, 0x74, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x23, 0x61, + 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x64, 0x4f, + 0x75, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x7f, 0x0a, 0x1e, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, + 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, + 0x52, 0x1b, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x79, 0x0a, 0x1c, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x66, 0x69, 0x72, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x15, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x72, 0x46, 0x69, 0x72, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x19, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x69, 0x72, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xae, 0x01, 0x0a, 0x2f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, + 0x79, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x5f, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, + 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, + 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, + 0x2a, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, + 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x98, 0x01, + 0x0a, 0x27, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, + 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, + 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, + 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x23, 0x61, 0x63, 0x74, 0x69, 0x76, + 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x82, 0x01, 0x0a, 0x1f, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, + 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, + 0x76, 0x31, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, + 0x1c, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x85, 0x01, 0x0a, 0x20, 0x6d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x72, 0x65, 0x63, 0x6f, 0x72, + 0x64, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, + 0x31, 0x2e, 0x4d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x65, 0x64, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, + 0x52, 0x1d, 0x6d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0xa7, 0x01, 0x0a, 0x2c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x65, 0x64, + 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x18, 0x1a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x28, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0xad, 0x01, 0x0a, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, + 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, + 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2a, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, + 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xbd, 0x01, 0x0a, 0x34, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, + 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x48, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, + 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0xa7, 0x01, 0x0a, 0x2c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, + 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x18, 0x1d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x28, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x42, 0x02, 0x68, 0x00, 0x12, 0xec, 0x01, 0x0a, 0x45, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, + 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x5f, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x57, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, + 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x3e, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x69, + 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xe3, 0x01, 0x0a, 0x42, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x5f, 0x65, 0x78, 0x74, 0x65, 0x72, + 0x6e, 0x61, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x1f, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x54, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, + 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x3b, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x61, + 0x6e, 0x63, 0x65, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0xd6, 0x01, 0x0a, 0x3d, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x18, 0x20, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x50, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, + 0x31, 0x2e, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x37, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0xbb, 0x01, 0x0a, 0x34, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, + 0x64, 0x5f, 0x61, 0x73, 0x5f, 0x6e, 0x65, 0x77, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x21, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x47, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, + 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, + 0x64, 0x41, 0x73, 0x4e, 0x65, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x64, + 0x41, 0x73, 0x4e, 0x65, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xca, 0x01, 0x0a, 0x39, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x5f, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, + 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x18, 0x22, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, + 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x69, 0x74, 0x69, + 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x48, 0x00, 0x52, 0x33, 0x73, 0x74, 0x61, 0x72, 0x74, 0x43, 0x68, 0x69, 0x6c, 0x64, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xc1, 0x01, 0x0a, 0x36, 0x73, + 0x74, 0x61, 0x72, 0x74, 0x5f, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x66, 0x61, 0x69, + 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x18, 0x23, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x49, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, + 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x48, 0x00, 0x52, 0x30, 0x73, 0x74, 0x61, 0x72, 0x74, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, + 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xb4, 0x01, 0x0a, 0x31, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x24, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x45, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, + 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x48, 0x00, 0x52, 0x2c, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, 0x74, + 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x42, 0x02, 0x68, 0x00, 0x12, 0xba, 0x01, 0x0a, 0x33, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, + 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x25, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x47, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, + 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, + 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2e, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, + 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xb1, 0x01, 0x0a, 0x30, 0x63, 0x68, 0x69, 0x6c, + 0x64, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x26, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x44, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, + 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, + 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2b, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x42, 0x02, 0x68, 0x00, 0x12, 0xb7, 0x01, 0x0a, 0x32, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x27, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x46, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, + 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, + 0x65, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x48, 0x00, 0x52, 0x2d, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, + 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xb8, 0x01, 0x0a, 0x33, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x64, 0x5f, 0x6f, 0x75, 0x74, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x28, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x46, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, + 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, + 0x69, 0x6d, 0x65, 0x64, 0x4f, 0x75, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2d, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, + 0x65, 0x64, 0x4f, 0x75, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xbd, 0x01, 0x0a, 0x34, 0x63, 0x68, 0x69, 0x6c, + 0x64, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x29, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x48, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x69, + 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2f, 0x63, 0x68, 0x69, + 0x6c, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xd6, + 0x01, 0x0a, 0x3d, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x5f, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x2a, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x50, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, + 0x61, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, + 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x48, 0x00, 0x52, 0x37, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xcd, 0x01, + 0x0a, 0x3a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x5f, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x2b, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x4d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, + 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x45, 0x78, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x34, 0x73, 0x69, + 0x67, 0x6e, 0x61, 0x6c, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0xc0, 0x01, 0x0a, 0x35, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x2c, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x49, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, + 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x30, 0x65, 0x78, 0x74, 0x65, 0x72, + 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xb7, 0x01, + 0x0a, 0x32, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x18, 0x2d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, + 0x2e, 0x55, 0x70, 0x73, 0x65, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x65, + 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2d, + 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x61, + 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xba, + 0x01, 0x0a, 0x33, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x65, + 0x70, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x18, 0x2e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x47, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, + 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x41, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, + 0x00, 0x52, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x41, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0xba, 0x01, 0x0a, 0x33, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, + 0x72, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x2f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x47, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, + 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x6a, + 0x65, 0x63, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x6a, + 0x65, 0x63, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xbd, 0x01, 0x0a, 0x34, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x75, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x30, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x48, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2f, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xc9, + 0x01, 0x0a, 0x38, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x65, + 0x72, 0x74, 0x69, 0x65, 0x73, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x5f, 0x65, 0x78, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x31, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4c, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, + 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x50, + 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, + 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x33, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x4d, 0x6f, 0x64, + 0x69, 0x66, 0x69, 0x65, 0x64, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x12, 0xc9, 0x01, 0x0a, 0x38, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x70, 0x72, 0x6f, + 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, + 0x5f, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x32, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x4c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, + 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, + 0x79, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x4d, 0x6f, 0x64, 0x69, 0x66, + 0x69, 0x65, 0x64, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x33, 0x61, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x4d, + 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x6c, 0x79, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0xaa, 0x01, 0x0a, 0x2d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, + 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x18, 0x33, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, + 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, + 0x69, 0x65, 0x73, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x29, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x4d, 0x6f, + 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xba, 0x01, 0x0a, 0x33, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x75, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x5f, 0x61, 0x64, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x34, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x47, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x2e, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x41, 0x64, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xa1, 0x01, 0x0a, 0x2a, + 0x6e, 0x65, 0x78, 0x75, 0x73, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x35, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, + 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x26, + 0x6e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x9b, 0x01, 0x0a, 0x28, 0x6e, 0x65, 0x78, + 0x75, 0x73, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x18, 0x36, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, + 0x31, 0x2e, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x24, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0xa1, + 0x01, 0x0a, 0x2a, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x37, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x65, 0x78, 0x75, 0x73, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, + 0x00, 0x52, 0x26, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x98, 0x01, 0x0a, 0x27, + 0x6e, 0x65, 0x78, 0x75, 0x73, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x66, + 0x61, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x38, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, + 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x23, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x9e, 0x01, 0x0a, 0x29, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x39, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, + 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, + 0x25, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, + 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x9f, 0x01, 0x0a, 0x2a, 0x6e, 0x65, 0x78, + 0x75, 0x73, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x64, 0x5f, 0x6f, 0x75, 0x74, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x3a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, + 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x64, 0x4f, 0x75, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x25, 0x6e, 0x65, 0x78, 0x75, 0x73, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x64, 0x4f, 0x75, 0x74, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0xb4, 0x01, 0x0a, 0x31, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x5f, 0x6f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x5f, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x3b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, + 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, + 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, + 0x00, 0x52, 0x2c, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x42, 0x0c, 0x0a, 0x0a, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x22, 0x4c, + 0x0a, 0x07, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x41, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, + 0x31, 0x2e, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x02, 0x68, 0x00, 0x42, 0x8e, 0x01, 0x0a, 0x1a, 0x69, 0x6f, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, + 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, + 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x25, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, + 0x2f, 0x76, 0x31, 0x3b, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0xaa, 0x02, 0x19, 0x54, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x48, 0x69, 0x73, 0x74, + 0x6f, 0x72, 0x79, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1c, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x3a, + 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_history_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_history_v1_message_proto_rawDescData = file_temporal_api_history_v1_message_proto_rawDesc +) + +func file_temporal_api_history_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_history_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_history_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_history_v1_message_proto_rawDescData) + }) + return file_temporal_api_history_v1_message_proto_rawDescData +} + +var file_temporal_api_history_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 58) +var file_temporal_api_history_v1_message_proto_goTypes = []any{ + (*WorkflowExecutionStartedEventAttributes)(nil), // 0: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes + (*WorkflowExecutionCompletedEventAttributes)(nil), // 1: temporal.api.history.v1.WorkflowExecutionCompletedEventAttributes + (*WorkflowExecutionFailedEventAttributes)(nil), // 2: temporal.api.history.v1.WorkflowExecutionFailedEventAttributes + (*WorkflowExecutionTimedOutEventAttributes)(nil), // 3: temporal.api.history.v1.WorkflowExecutionTimedOutEventAttributes + (*WorkflowExecutionContinuedAsNewEventAttributes)(nil), // 4: temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes + (*WorkflowTaskScheduledEventAttributes)(nil), // 5: temporal.api.history.v1.WorkflowTaskScheduledEventAttributes + (*WorkflowTaskStartedEventAttributes)(nil), // 6: temporal.api.history.v1.WorkflowTaskStartedEventAttributes + (*WorkflowTaskCompletedEventAttributes)(nil), // 7: temporal.api.history.v1.WorkflowTaskCompletedEventAttributes + (*WorkflowTaskTimedOutEventAttributes)(nil), // 8: temporal.api.history.v1.WorkflowTaskTimedOutEventAttributes + (*WorkflowTaskFailedEventAttributes)(nil), // 9: temporal.api.history.v1.WorkflowTaskFailedEventAttributes + (*ActivityTaskScheduledEventAttributes)(nil), // 10: temporal.api.history.v1.ActivityTaskScheduledEventAttributes + (*ActivityTaskStartedEventAttributes)(nil), // 11: temporal.api.history.v1.ActivityTaskStartedEventAttributes + (*ActivityTaskCompletedEventAttributes)(nil), // 12: temporal.api.history.v1.ActivityTaskCompletedEventAttributes + (*ActivityTaskFailedEventAttributes)(nil), // 13: temporal.api.history.v1.ActivityTaskFailedEventAttributes + (*ActivityTaskTimedOutEventAttributes)(nil), // 14: temporal.api.history.v1.ActivityTaskTimedOutEventAttributes + (*ActivityTaskCancelRequestedEventAttributes)(nil), // 15: temporal.api.history.v1.ActivityTaskCancelRequestedEventAttributes + (*ActivityTaskCanceledEventAttributes)(nil), // 16: temporal.api.history.v1.ActivityTaskCanceledEventAttributes + (*TimerStartedEventAttributes)(nil), // 17: temporal.api.history.v1.TimerStartedEventAttributes + (*TimerFiredEventAttributes)(nil), // 18: temporal.api.history.v1.TimerFiredEventAttributes + (*TimerCanceledEventAttributes)(nil), // 19: temporal.api.history.v1.TimerCanceledEventAttributes + (*WorkflowExecutionCancelRequestedEventAttributes)(nil), // 20: temporal.api.history.v1.WorkflowExecutionCancelRequestedEventAttributes + (*WorkflowExecutionCanceledEventAttributes)(nil), // 21: temporal.api.history.v1.WorkflowExecutionCanceledEventAttributes + (*MarkerRecordedEventAttributes)(nil), // 22: temporal.api.history.v1.MarkerRecordedEventAttributes + (*WorkflowExecutionSignaledEventAttributes)(nil), // 23: temporal.api.history.v1.WorkflowExecutionSignaledEventAttributes + (*WorkflowExecutionTerminatedEventAttributes)(nil), // 24: temporal.api.history.v1.WorkflowExecutionTerminatedEventAttributes + (*RequestCancelExternalWorkflowExecutionInitiatedEventAttributes)(nil), // 25: temporal.api.history.v1.RequestCancelExternalWorkflowExecutionInitiatedEventAttributes + (*RequestCancelExternalWorkflowExecutionFailedEventAttributes)(nil), // 26: temporal.api.history.v1.RequestCancelExternalWorkflowExecutionFailedEventAttributes + (*ExternalWorkflowExecutionCancelRequestedEventAttributes)(nil), // 27: temporal.api.history.v1.ExternalWorkflowExecutionCancelRequestedEventAttributes + (*SignalExternalWorkflowExecutionInitiatedEventAttributes)(nil), // 28: temporal.api.history.v1.SignalExternalWorkflowExecutionInitiatedEventAttributes + (*SignalExternalWorkflowExecutionFailedEventAttributes)(nil), // 29: temporal.api.history.v1.SignalExternalWorkflowExecutionFailedEventAttributes + (*ExternalWorkflowExecutionSignaledEventAttributes)(nil), // 30: temporal.api.history.v1.ExternalWorkflowExecutionSignaledEventAttributes + (*UpsertWorkflowSearchAttributesEventAttributes)(nil), // 31: temporal.api.history.v1.UpsertWorkflowSearchAttributesEventAttributes + (*WorkflowPropertiesModifiedEventAttributes)(nil), // 32: temporal.api.history.v1.WorkflowPropertiesModifiedEventAttributes + (*StartChildWorkflowExecutionInitiatedEventAttributes)(nil), // 33: temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes + (*StartChildWorkflowExecutionFailedEventAttributes)(nil), // 34: temporal.api.history.v1.StartChildWorkflowExecutionFailedEventAttributes + (*ChildWorkflowExecutionStartedEventAttributes)(nil), // 35: temporal.api.history.v1.ChildWorkflowExecutionStartedEventAttributes + (*ChildWorkflowExecutionCompletedEventAttributes)(nil), // 36: temporal.api.history.v1.ChildWorkflowExecutionCompletedEventAttributes + (*ChildWorkflowExecutionFailedEventAttributes)(nil), // 37: temporal.api.history.v1.ChildWorkflowExecutionFailedEventAttributes + (*ChildWorkflowExecutionCanceledEventAttributes)(nil), // 38: temporal.api.history.v1.ChildWorkflowExecutionCanceledEventAttributes + (*ChildWorkflowExecutionTimedOutEventAttributes)(nil), // 39: temporal.api.history.v1.ChildWorkflowExecutionTimedOutEventAttributes + (*ChildWorkflowExecutionTerminatedEventAttributes)(nil), // 40: temporal.api.history.v1.ChildWorkflowExecutionTerminatedEventAttributes + (*WorkflowPropertiesModifiedExternallyEventAttributes)(nil), // 41: temporal.api.history.v1.WorkflowPropertiesModifiedExternallyEventAttributes + (*ActivityPropertiesModifiedExternallyEventAttributes)(nil), // 42: temporal.api.history.v1.ActivityPropertiesModifiedExternallyEventAttributes + (*WorkflowExecutionUpdateAcceptedEventAttributes)(nil), // 43: temporal.api.history.v1.WorkflowExecutionUpdateAcceptedEventAttributes + (*WorkflowExecutionUpdateCompletedEventAttributes)(nil), // 44: temporal.api.history.v1.WorkflowExecutionUpdateCompletedEventAttributes + (*WorkflowExecutionUpdateRejectedEventAttributes)(nil), // 45: temporal.api.history.v1.WorkflowExecutionUpdateRejectedEventAttributes + (*WorkflowExecutionUpdateAdmittedEventAttributes)(nil), // 46: temporal.api.history.v1.WorkflowExecutionUpdateAdmittedEventAttributes + (*NexusOperationScheduledEventAttributes)(nil), // 47: temporal.api.history.v1.NexusOperationScheduledEventAttributes + (*NexusOperationStartedEventAttributes)(nil), // 48: temporal.api.history.v1.NexusOperationStartedEventAttributes + (*NexusOperationCompletedEventAttributes)(nil), // 49: temporal.api.history.v1.NexusOperationCompletedEventAttributes + (*NexusOperationFailedEventAttributes)(nil), // 50: temporal.api.history.v1.NexusOperationFailedEventAttributes + (*NexusOperationTimedOutEventAttributes)(nil), // 51: temporal.api.history.v1.NexusOperationTimedOutEventAttributes + (*NexusOperationCanceledEventAttributes)(nil), // 52: temporal.api.history.v1.NexusOperationCanceledEventAttributes + (*NexusOperationCancelRequestedEventAttributes)(nil), // 53: temporal.api.history.v1.NexusOperationCancelRequestedEventAttributes + (*HistoryEvent)(nil), // 54: temporal.api.history.v1.HistoryEvent + (*History)(nil), // 55: temporal.api.history.v1.History + nil, // 56: temporal.api.history.v1.MarkerRecordedEventAttributes.DetailsEntry + nil, // 57: temporal.api.history.v1.NexusOperationScheduledEventAttributes.NexusHeaderEntry + (*v1.WorkflowType)(nil), // 58: temporal.api.common.v1.WorkflowType + (*v1.WorkflowExecution)(nil), // 59: temporal.api.common.v1.WorkflowExecution + (*v11.TaskQueue)(nil), // 60: temporal.api.taskqueue.v1.TaskQueue + (*v1.Payloads)(nil), // 61: temporal.api.common.v1.Payloads + (*durationpb.Duration)(nil), // 62: google.protobuf.Duration + (v12.ContinueAsNewInitiator)(0), // 63: temporal.api.enums.v1.ContinueAsNewInitiator + (*v13.Failure)(nil), // 64: temporal.api.failure.v1.Failure + (*v1.RetryPolicy)(nil), // 65: temporal.api.common.v1.RetryPolicy + (*timestamppb.Timestamp)(nil), // 66: google.protobuf.Timestamp + (*v1.Memo)(nil), // 67: temporal.api.common.v1.Memo + (*v1.SearchAttributes)(nil), // 68: temporal.api.common.v1.SearchAttributes + (*v14.ResetPoints)(nil), // 69: temporal.api.workflow.v1.ResetPoints + (*v1.Header)(nil), // 70: temporal.api.common.v1.Header + (*v1.WorkerVersionStamp)(nil), // 71: temporal.api.common.v1.WorkerVersionStamp + (*v1.Callback)(nil), // 72: temporal.api.common.v1.Callback + (v12.RetryState)(0), // 73: temporal.api.enums.v1.RetryState + (*v15.WorkflowTaskCompletedMetadata)(nil), // 74: temporal.api.sdk.v1.WorkflowTaskCompletedMetadata + (*v1.MeteringMetadata)(nil), // 75: temporal.api.common.v1.MeteringMetadata + (v12.TimeoutType)(0), // 76: temporal.api.enums.v1.TimeoutType + (v12.WorkflowTaskFailedCause)(0), // 77: temporal.api.enums.v1.WorkflowTaskFailedCause + (*v1.ActivityType)(nil), // 78: temporal.api.common.v1.ActivityType + (v12.CancelExternalWorkflowExecutionFailedCause)(0), // 79: temporal.api.enums.v1.CancelExternalWorkflowExecutionFailedCause + (v12.SignalExternalWorkflowExecutionFailedCause)(0), // 80: temporal.api.enums.v1.SignalExternalWorkflowExecutionFailedCause + (v12.ParentClosePolicy)(0), // 81: temporal.api.enums.v1.ParentClosePolicy + (v12.WorkflowIdReusePolicy)(0), // 82: temporal.api.enums.v1.WorkflowIdReusePolicy + (v12.StartChildWorkflowExecutionFailedCause)(0), // 83: temporal.api.enums.v1.StartChildWorkflowExecutionFailedCause + (*v16.Request)(nil), // 84: temporal.api.update.v1.Request + (*v16.Meta)(nil), // 85: temporal.api.update.v1.Meta + (*v16.Outcome)(nil), // 86: temporal.api.update.v1.Outcome + (v12.UpdateAdmittedEventOrigin)(0), // 87: temporal.api.enums.v1.UpdateAdmittedEventOrigin + (*v1.Payload)(nil), // 88: temporal.api.common.v1.Payload + (v12.EventType)(0), // 89: temporal.api.enums.v1.EventType + (*v15.UserMetadata)(nil), // 90: temporal.api.sdk.v1.UserMetadata + (*v1.Link)(nil), // 91: temporal.api.common.v1.Link +} +var file_temporal_api_history_v1_message_proto_depIdxs = []int32{ + 58, // 0: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 59, // 1: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.parent_workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 60, // 2: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 61, // 3: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.input:type_name -> temporal.api.common.v1.Payloads + 62, // 4: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.workflow_execution_timeout:type_name -> google.protobuf.Duration + 62, // 5: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.workflow_run_timeout:type_name -> google.protobuf.Duration + 62, // 6: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.workflow_task_timeout:type_name -> google.protobuf.Duration + 63, // 7: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.initiator:type_name -> temporal.api.enums.v1.ContinueAsNewInitiator + 64, // 8: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.continued_failure:type_name -> temporal.api.failure.v1.Failure + 61, // 9: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.last_completion_result:type_name -> temporal.api.common.v1.Payloads + 65, // 10: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.retry_policy:type_name -> temporal.api.common.v1.RetryPolicy + 66, // 11: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.workflow_execution_expiration_time:type_name -> google.protobuf.Timestamp + 62, // 12: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.first_workflow_task_backoff:type_name -> google.protobuf.Duration + 67, // 13: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.memo:type_name -> temporal.api.common.v1.Memo + 68, // 14: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 69, // 15: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.prev_auto_reset_points:type_name -> temporal.api.workflow.v1.ResetPoints + 70, // 16: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.header:type_name -> temporal.api.common.v1.Header + 71, // 17: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.source_version_stamp:type_name -> temporal.api.common.v1.WorkerVersionStamp + 72, // 18: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.completion_callbacks:type_name -> temporal.api.common.v1.Callback + 59, // 19: temporal.api.history.v1.WorkflowExecutionStartedEventAttributes.root_workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 61, // 20: temporal.api.history.v1.WorkflowExecutionCompletedEventAttributes.result:type_name -> temporal.api.common.v1.Payloads + 64, // 21: temporal.api.history.v1.WorkflowExecutionFailedEventAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 73, // 22: temporal.api.history.v1.WorkflowExecutionFailedEventAttributes.retry_state:type_name -> temporal.api.enums.v1.RetryState + 73, // 23: temporal.api.history.v1.WorkflowExecutionTimedOutEventAttributes.retry_state:type_name -> temporal.api.enums.v1.RetryState + 58, // 24: temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 60, // 25: temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 61, // 26: temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes.input:type_name -> temporal.api.common.v1.Payloads + 62, // 27: temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes.workflow_run_timeout:type_name -> google.protobuf.Duration + 62, // 28: temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes.workflow_task_timeout:type_name -> google.protobuf.Duration + 62, // 29: temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes.backoff_start_interval:type_name -> google.protobuf.Duration + 63, // 30: temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes.initiator:type_name -> temporal.api.enums.v1.ContinueAsNewInitiator + 64, // 31: temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 61, // 32: temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes.last_completion_result:type_name -> temporal.api.common.v1.Payloads + 70, // 33: temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes.header:type_name -> temporal.api.common.v1.Header + 67, // 34: temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes.memo:type_name -> temporal.api.common.v1.Memo + 68, // 35: temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 60, // 36: temporal.api.history.v1.WorkflowTaskScheduledEventAttributes.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 62, // 37: temporal.api.history.v1.WorkflowTaskScheduledEventAttributes.start_to_close_timeout:type_name -> google.protobuf.Duration + 71, // 38: temporal.api.history.v1.WorkflowTaskStartedEventAttributes.worker_version:type_name -> temporal.api.common.v1.WorkerVersionStamp + 71, // 39: temporal.api.history.v1.WorkflowTaskCompletedEventAttributes.worker_version:type_name -> temporal.api.common.v1.WorkerVersionStamp + 74, // 40: temporal.api.history.v1.WorkflowTaskCompletedEventAttributes.sdk_metadata:type_name -> temporal.api.sdk.v1.WorkflowTaskCompletedMetadata + 75, // 41: temporal.api.history.v1.WorkflowTaskCompletedEventAttributes.metering_metadata:type_name -> temporal.api.common.v1.MeteringMetadata + 76, // 42: temporal.api.history.v1.WorkflowTaskTimedOutEventAttributes.timeout_type:type_name -> temporal.api.enums.v1.TimeoutType + 77, // 43: temporal.api.history.v1.WorkflowTaskFailedEventAttributes.cause:type_name -> temporal.api.enums.v1.WorkflowTaskFailedCause + 64, // 44: temporal.api.history.v1.WorkflowTaskFailedEventAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 71, // 45: temporal.api.history.v1.WorkflowTaskFailedEventAttributes.worker_version:type_name -> temporal.api.common.v1.WorkerVersionStamp + 78, // 46: temporal.api.history.v1.ActivityTaskScheduledEventAttributes.activity_type:type_name -> temporal.api.common.v1.ActivityType + 60, // 47: temporal.api.history.v1.ActivityTaskScheduledEventAttributes.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 70, // 48: temporal.api.history.v1.ActivityTaskScheduledEventAttributes.header:type_name -> temporal.api.common.v1.Header + 61, // 49: temporal.api.history.v1.ActivityTaskScheduledEventAttributes.input:type_name -> temporal.api.common.v1.Payloads + 62, // 50: temporal.api.history.v1.ActivityTaskScheduledEventAttributes.schedule_to_close_timeout:type_name -> google.protobuf.Duration + 62, // 51: temporal.api.history.v1.ActivityTaskScheduledEventAttributes.schedule_to_start_timeout:type_name -> google.protobuf.Duration + 62, // 52: temporal.api.history.v1.ActivityTaskScheduledEventAttributes.start_to_close_timeout:type_name -> google.protobuf.Duration + 62, // 53: temporal.api.history.v1.ActivityTaskScheduledEventAttributes.heartbeat_timeout:type_name -> google.protobuf.Duration + 65, // 54: temporal.api.history.v1.ActivityTaskScheduledEventAttributes.retry_policy:type_name -> temporal.api.common.v1.RetryPolicy + 64, // 55: temporal.api.history.v1.ActivityTaskStartedEventAttributes.last_failure:type_name -> temporal.api.failure.v1.Failure + 71, // 56: temporal.api.history.v1.ActivityTaskStartedEventAttributes.worker_version:type_name -> temporal.api.common.v1.WorkerVersionStamp + 61, // 57: temporal.api.history.v1.ActivityTaskCompletedEventAttributes.result:type_name -> temporal.api.common.v1.Payloads + 71, // 58: temporal.api.history.v1.ActivityTaskCompletedEventAttributes.worker_version:type_name -> temporal.api.common.v1.WorkerVersionStamp + 64, // 59: temporal.api.history.v1.ActivityTaskFailedEventAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 73, // 60: temporal.api.history.v1.ActivityTaskFailedEventAttributes.retry_state:type_name -> temporal.api.enums.v1.RetryState + 71, // 61: temporal.api.history.v1.ActivityTaskFailedEventAttributes.worker_version:type_name -> temporal.api.common.v1.WorkerVersionStamp + 64, // 62: temporal.api.history.v1.ActivityTaskTimedOutEventAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 73, // 63: temporal.api.history.v1.ActivityTaskTimedOutEventAttributes.retry_state:type_name -> temporal.api.enums.v1.RetryState + 61, // 64: temporal.api.history.v1.ActivityTaskCanceledEventAttributes.details:type_name -> temporal.api.common.v1.Payloads + 71, // 65: temporal.api.history.v1.ActivityTaskCanceledEventAttributes.worker_version:type_name -> temporal.api.common.v1.WorkerVersionStamp + 62, // 66: temporal.api.history.v1.TimerStartedEventAttributes.start_to_fire_timeout:type_name -> google.protobuf.Duration + 59, // 67: temporal.api.history.v1.WorkflowExecutionCancelRequestedEventAttributes.external_workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 61, // 68: temporal.api.history.v1.WorkflowExecutionCanceledEventAttributes.details:type_name -> temporal.api.common.v1.Payloads + 56, // 69: temporal.api.history.v1.MarkerRecordedEventAttributes.details:type_name -> temporal.api.history.v1.MarkerRecordedEventAttributes.DetailsEntry + 70, // 70: temporal.api.history.v1.MarkerRecordedEventAttributes.header:type_name -> temporal.api.common.v1.Header + 64, // 71: temporal.api.history.v1.MarkerRecordedEventAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 61, // 72: temporal.api.history.v1.WorkflowExecutionSignaledEventAttributes.input:type_name -> temporal.api.common.v1.Payloads + 70, // 73: temporal.api.history.v1.WorkflowExecutionSignaledEventAttributes.header:type_name -> temporal.api.common.v1.Header + 59, // 74: temporal.api.history.v1.WorkflowExecutionSignaledEventAttributes.external_workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 61, // 75: temporal.api.history.v1.WorkflowExecutionTerminatedEventAttributes.details:type_name -> temporal.api.common.v1.Payloads + 59, // 76: temporal.api.history.v1.RequestCancelExternalWorkflowExecutionInitiatedEventAttributes.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 79, // 77: temporal.api.history.v1.RequestCancelExternalWorkflowExecutionFailedEventAttributes.cause:type_name -> temporal.api.enums.v1.CancelExternalWorkflowExecutionFailedCause + 59, // 78: temporal.api.history.v1.RequestCancelExternalWorkflowExecutionFailedEventAttributes.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 59, // 79: temporal.api.history.v1.ExternalWorkflowExecutionCancelRequestedEventAttributes.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 59, // 80: temporal.api.history.v1.SignalExternalWorkflowExecutionInitiatedEventAttributes.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 61, // 81: temporal.api.history.v1.SignalExternalWorkflowExecutionInitiatedEventAttributes.input:type_name -> temporal.api.common.v1.Payloads + 70, // 82: temporal.api.history.v1.SignalExternalWorkflowExecutionInitiatedEventAttributes.header:type_name -> temporal.api.common.v1.Header + 80, // 83: temporal.api.history.v1.SignalExternalWorkflowExecutionFailedEventAttributes.cause:type_name -> temporal.api.enums.v1.SignalExternalWorkflowExecutionFailedCause + 59, // 84: temporal.api.history.v1.SignalExternalWorkflowExecutionFailedEventAttributes.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 59, // 85: temporal.api.history.v1.ExternalWorkflowExecutionSignaledEventAttributes.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 68, // 86: temporal.api.history.v1.UpsertWorkflowSearchAttributesEventAttributes.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 67, // 87: temporal.api.history.v1.WorkflowPropertiesModifiedEventAttributes.upserted_memo:type_name -> temporal.api.common.v1.Memo + 58, // 88: temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 60, // 89: temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 61, // 90: temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes.input:type_name -> temporal.api.common.v1.Payloads + 62, // 91: temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes.workflow_execution_timeout:type_name -> google.protobuf.Duration + 62, // 92: temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes.workflow_run_timeout:type_name -> google.protobuf.Duration + 62, // 93: temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes.workflow_task_timeout:type_name -> google.protobuf.Duration + 81, // 94: temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes.parent_close_policy:type_name -> temporal.api.enums.v1.ParentClosePolicy + 82, // 95: temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes.workflow_id_reuse_policy:type_name -> temporal.api.enums.v1.WorkflowIdReusePolicy + 65, // 96: temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes.retry_policy:type_name -> temporal.api.common.v1.RetryPolicy + 70, // 97: temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes.header:type_name -> temporal.api.common.v1.Header + 67, // 98: temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes.memo:type_name -> temporal.api.common.v1.Memo + 68, // 99: temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 58, // 100: temporal.api.history.v1.StartChildWorkflowExecutionFailedEventAttributes.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 83, // 101: temporal.api.history.v1.StartChildWorkflowExecutionFailedEventAttributes.cause:type_name -> temporal.api.enums.v1.StartChildWorkflowExecutionFailedCause + 59, // 102: temporal.api.history.v1.ChildWorkflowExecutionStartedEventAttributes.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 58, // 103: temporal.api.history.v1.ChildWorkflowExecutionStartedEventAttributes.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 70, // 104: temporal.api.history.v1.ChildWorkflowExecutionStartedEventAttributes.header:type_name -> temporal.api.common.v1.Header + 61, // 105: temporal.api.history.v1.ChildWorkflowExecutionCompletedEventAttributes.result:type_name -> temporal.api.common.v1.Payloads + 59, // 106: temporal.api.history.v1.ChildWorkflowExecutionCompletedEventAttributes.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 58, // 107: temporal.api.history.v1.ChildWorkflowExecutionCompletedEventAttributes.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 64, // 108: temporal.api.history.v1.ChildWorkflowExecutionFailedEventAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 59, // 109: temporal.api.history.v1.ChildWorkflowExecutionFailedEventAttributes.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 58, // 110: temporal.api.history.v1.ChildWorkflowExecutionFailedEventAttributes.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 73, // 111: temporal.api.history.v1.ChildWorkflowExecutionFailedEventAttributes.retry_state:type_name -> temporal.api.enums.v1.RetryState + 61, // 112: temporal.api.history.v1.ChildWorkflowExecutionCanceledEventAttributes.details:type_name -> temporal.api.common.v1.Payloads + 59, // 113: temporal.api.history.v1.ChildWorkflowExecutionCanceledEventAttributes.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 58, // 114: temporal.api.history.v1.ChildWorkflowExecutionCanceledEventAttributes.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 59, // 115: temporal.api.history.v1.ChildWorkflowExecutionTimedOutEventAttributes.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 58, // 116: temporal.api.history.v1.ChildWorkflowExecutionTimedOutEventAttributes.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 73, // 117: temporal.api.history.v1.ChildWorkflowExecutionTimedOutEventAttributes.retry_state:type_name -> temporal.api.enums.v1.RetryState + 59, // 118: temporal.api.history.v1.ChildWorkflowExecutionTerminatedEventAttributes.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 58, // 119: temporal.api.history.v1.ChildWorkflowExecutionTerminatedEventAttributes.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 62, // 120: temporal.api.history.v1.WorkflowPropertiesModifiedExternallyEventAttributes.new_workflow_task_timeout:type_name -> google.protobuf.Duration + 62, // 121: temporal.api.history.v1.WorkflowPropertiesModifiedExternallyEventAttributes.new_workflow_run_timeout:type_name -> google.protobuf.Duration + 62, // 122: temporal.api.history.v1.WorkflowPropertiesModifiedExternallyEventAttributes.new_workflow_execution_timeout:type_name -> google.protobuf.Duration + 67, // 123: temporal.api.history.v1.WorkflowPropertiesModifiedExternallyEventAttributes.upserted_memo:type_name -> temporal.api.common.v1.Memo + 65, // 124: temporal.api.history.v1.ActivityPropertiesModifiedExternallyEventAttributes.new_retry_policy:type_name -> temporal.api.common.v1.RetryPolicy + 84, // 125: temporal.api.history.v1.WorkflowExecutionUpdateAcceptedEventAttributes.accepted_request:type_name -> temporal.api.update.v1.Request + 85, // 126: temporal.api.history.v1.WorkflowExecutionUpdateCompletedEventAttributes.meta:type_name -> temporal.api.update.v1.Meta + 86, // 127: temporal.api.history.v1.WorkflowExecutionUpdateCompletedEventAttributes.outcome:type_name -> temporal.api.update.v1.Outcome + 84, // 128: temporal.api.history.v1.WorkflowExecutionUpdateRejectedEventAttributes.rejected_request:type_name -> temporal.api.update.v1.Request + 64, // 129: temporal.api.history.v1.WorkflowExecutionUpdateRejectedEventAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 84, // 130: temporal.api.history.v1.WorkflowExecutionUpdateAdmittedEventAttributes.request:type_name -> temporal.api.update.v1.Request + 87, // 131: temporal.api.history.v1.WorkflowExecutionUpdateAdmittedEventAttributes.origin:type_name -> temporal.api.enums.v1.UpdateAdmittedEventOrigin + 88, // 132: temporal.api.history.v1.NexusOperationScheduledEventAttributes.input:type_name -> temporal.api.common.v1.Payload + 62, // 133: temporal.api.history.v1.NexusOperationScheduledEventAttributes.schedule_to_close_timeout:type_name -> google.protobuf.Duration + 57, // 134: temporal.api.history.v1.NexusOperationScheduledEventAttributes.nexus_header:type_name -> temporal.api.history.v1.NexusOperationScheduledEventAttributes.NexusHeaderEntry + 88, // 135: temporal.api.history.v1.NexusOperationCompletedEventAttributes.result:type_name -> temporal.api.common.v1.Payload + 64, // 136: temporal.api.history.v1.NexusOperationFailedEventAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 64, // 137: temporal.api.history.v1.NexusOperationTimedOutEventAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 64, // 138: temporal.api.history.v1.NexusOperationCanceledEventAttributes.failure:type_name -> temporal.api.failure.v1.Failure + 66, // 139: temporal.api.history.v1.HistoryEvent.event_time:type_name -> google.protobuf.Timestamp + 89, // 140: temporal.api.history.v1.HistoryEvent.event_type:type_name -> temporal.api.enums.v1.EventType + 90, // 141: temporal.api.history.v1.HistoryEvent.user_metadata:type_name -> temporal.api.sdk.v1.UserMetadata + 91, // 142: temporal.api.history.v1.HistoryEvent.links:type_name -> temporal.api.common.v1.Link + 0, // 143: temporal.api.history.v1.HistoryEvent.workflow_execution_started_event_attributes:type_name -> temporal.api.history.v1.WorkflowExecutionStartedEventAttributes + 1, // 144: temporal.api.history.v1.HistoryEvent.workflow_execution_completed_event_attributes:type_name -> temporal.api.history.v1.WorkflowExecutionCompletedEventAttributes + 2, // 145: temporal.api.history.v1.HistoryEvent.workflow_execution_failed_event_attributes:type_name -> temporal.api.history.v1.WorkflowExecutionFailedEventAttributes + 3, // 146: temporal.api.history.v1.HistoryEvent.workflow_execution_timed_out_event_attributes:type_name -> temporal.api.history.v1.WorkflowExecutionTimedOutEventAttributes + 5, // 147: temporal.api.history.v1.HistoryEvent.workflow_task_scheduled_event_attributes:type_name -> temporal.api.history.v1.WorkflowTaskScheduledEventAttributes + 6, // 148: temporal.api.history.v1.HistoryEvent.workflow_task_started_event_attributes:type_name -> temporal.api.history.v1.WorkflowTaskStartedEventAttributes + 7, // 149: temporal.api.history.v1.HistoryEvent.workflow_task_completed_event_attributes:type_name -> temporal.api.history.v1.WorkflowTaskCompletedEventAttributes + 8, // 150: temporal.api.history.v1.HistoryEvent.workflow_task_timed_out_event_attributes:type_name -> temporal.api.history.v1.WorkflowTaskTimedOutEventAttributes + 9, // 151: temporal.api.history.v1.HistoryEvent.workflow_task_failed_event_attributes:type_name -> temporal.api.history.v1.WorkflowTaskFailedEventAttributes + 10, // 152: temporal.api.history.v1.HistoryEvent.activity_task_scheduled_event_attributes:type_name -> temporal.api.history.v1.ActivityTaskScheduledEventAttributes + 11, // 153: temporal.api.history.v1.HistoryEvent.activity_task_started_event_attributes:type_name -> temporal.api.history.v1.ActivityTaskStartedEventAttributes + 12, // 154: temporal.api.history.v1.HistoryEvent.activity_task_completed_event_attributes:type_name -> temporal.api.history.v1.ActivityTaskCompletedEventAttributes + 13, // 155: temporal.api.history.v1.HistoryEvent.activity_task_failed_event_attributes:type_name -> temporal.api.history.v1.ActivityTaskFailedEventAttributes + 14, // 156: temporal.api.history.v1.HistoryEvent.activity_task_timed_out_event_attributes:type_name -> temporal.api.history.v1.ActivityTaskTimedOutEventAttributes + 17, // 157: temporal.api.history.v1.HistoryEvent.timer_started_event_attributes:type_name -> temporal.api.history.v1.TimerStartedEventAttributes + 18, // 158: temporal.api.history.v1.HistoryEvent.timer_fired_event_attributes:type_name -> temporal.api.history.v1.TimerFiredEventAttributes + 15, // 159: temporal.api.history.v1.HistoryEvent.activity_task_cancel_requested_event_attributes:type_name -> temporal.api.history.v1.ActivityTaskCancelRequestedEventAttributes + 16, // 160: temporal.api.history.v1.HistoryEvent.activity_task_canceled_event_attributes:type_name -> temporal.api.history.v1.ActivityTaskCanceledEventAttributes + 19, // 161: temporal.api.history.v1.HistoryEvent.timer_canceled_event_attributes:type_name -> temporal.api.history.v1.TimerCanceledEventAttributes + 22, // 162: temporal.api.history.v1.HistoryEvent.marker_recorded_event_attributes:type_name -> temporal.api.history.v1.MarkerRecordedEventAttributes + 23, // 163: temporal.api.history.v1.HistoryEvent.workflow_execution_signaled_event_attributes:type_name -> temporal.api.history.v1.WorkflowExecutionSignaledEventAttributes + 24, // 164: temporal.api.history.v1.HistoryEvent.workflow_execution_terminated_event_attributes:type_name -> temporal.api.history.v1.WorkflowExecutionTerminatedEventAttributes + 20, // 165: temporal.api.history.v1.HistoryEvent.workflow_execution_cancel_requested_event_attributes:type_name -> temporal.api.history.v1.WorkflowExecutionCancelRequestedEventAttributes + 21, // 166: temporal.api.history.v1.HistoryEvent.workflow_execution_canceled_event_attributes:type_name -> temporal.api.history.v1.WorkflowExecutionCanceledEventAttributes + 25, // 167: temporal.api.history.v1.HistoryEvent.request_cancel_external_workflow_execution_initiated_event_attributes:type_name -> temporal.api.history.v1.RequestCancelExternalWorkflowExecutionInitiatedEventAttributes + 26, // 168: temporal.api.history.v1.HistoryEvent.request_cancel_external_workflow_execution_failed_event_attributes:type_name -> temporal.api.history.v1.RequestCancelExternalWorkflowExecutionFailedEventAttributes + 27, // 169: temporal.api.history.v1.HistoryEvent.external_workflow_execution_cancel_requested_event_attributes:type_name -> temporal.api.history.v1.ExternalWorkflowExecutionCancelRequestedEventAttributes + 4, // 170: temporal.api.history.v1.HistoryEvent.workflow_execution_continued_as_new_event_attributes:type_name -> temporal.api.history.v1.WorkflowExecutionContinuedAsNewEventAttributes + 33, // 171: temporal.api.history.v1.HistoryEvent.start_child_workflow_execution_initiated_event_attributes:type_name -> temporal.api.history.v1.StartChildWorkflowExecutionInitiatedEventAttributes + 34, // 172: temporal.api.history.v1.HistoryEvent.start_child_workflow_execution_failed_event_attributes:type_name -> temporal.api.history.v1.StartChildWorkflowExecutionFailedEventAttributes + 35, // 173: temporal.api.history.v1.HistoryEvent.child_workflow_execution_started_event_attributes:type_name -> temporal.api.history.v1.ChildWorkflowExecutionStartedEventAttributes + 36, // 174: temporal.api.history.v1.HistoryEvent.child_workflow_execution_completed_event_attributes:type_name -> temporal.api.history.v1.ChildWorkflowExecutionCompletedEventAttributes + 37, // 175: temporal.api.history.v1.HistoryEvent.child_workflow_execution_failed_event_attributes:type_name -> temporal.api.history.v1.ChildWorkflowExecutionFailedEventAttributes + 38, // 176: temporal.api.history.v1.HistoryEvent.child_workflow_execution_canceled_event_attributes:type_name -> temporal.api.history.v1.ChildWorkflowExecutionCanceledEventAttributes + 39, // 177: temporal.api.history.v1.HistoryEvent.child_workflow_execution_timed_out_event_attributes:type_name -> temporal.api.history.v1.ChildWorkflowExecutionTimedOutEventAttributes + 40, // 178: temporal.api.history.v1.HistoryEvent.child_workflow_execution_terminated_event_attributes:type_name -> temporal.api.history.v1.ChildWorkflowExecutionTerminatedEventAttributes + 28, // 179: temporal.api.history.v1.HistoryEvent.signal_external_workflow_execution_initiated_event_attributes:type_name -> temporal.api.history.v1.SignalExternalWorkflowExecutionInitiatedEventAttributes + 29, // 180: temporal.api.history.v1.HistoryEvent.signal_external_workflow_execution_failed_event_attributes:type_name -> temporal.api.history.v1.SignalExternalWorkflowExecutionFailedEventAttributes + 30, // 181: temporal.api.history.v1.HistoryEvent.external_workflow_execution_signaled_event_attributes:type_name -> temporal.api.history.v1.ExternalWorkflowExecutionSignaledEventAttributes + 31, // 182: temporal.api.history.v1.HistoryEvent.upsert_workflow_search_attributes_event_attributes:type_name -> temporal.api.history.v1.UpsertWorkflowSearchAttributesEventAttributes + 43, // 183: temporal.api.history.v1.HistoryEvent.workflow_execution_update_accepted_event_attributes:type_name -> temporal.api.history.v1.WorkflowExecutionUpdateAcceptedEventAttributes + 45, // 184: temporal.api.history.v1.HistoryEvent.workflow_execution_update_rejected_event_attributes:type_name -> temporal.api.history.v1.WorkflowExecutionUpdateRejectedEventAttributes + 44, // 185: temporal.api.history.v1.HistoryEvent.workflow_execution_update_completed_event_attributes:type_name -> temporal.api.history.v1.WorkflowExecutionUpdateCompletedEventAttributes + 41, // 186: temporal.api.history.v1.HistoryEvent.workflow_properties_modified_externally_event_attributes:type_name -> temporal.api.history.v1.WorkflowPropertiesModifiedExternallyEventAttributes + 42, // 187: temporal.api.history.v1.HistoryEvent.activity_properties_modified_externally_event_attributes:type_name -> temporal.api.history.v1.ActivityPropertiesModifiedExternallyEventAttributes + 32, // 188: temporal.api.history.v1.HistoryEvent.workflow_properties_modified_event_attributes:type_name -> temporal.api.history.v1.WorkflowPropertiesModifiedEventAttributes + 46, // 189: temporal.api.history.v1.HistoryEvent.workflow_execution_update_admitted_event_attributes:type_name -> temporal.api.history.v1.WorkflowExecutionUpdateAdmittedEventAttributes + 47, // 190: temporal.api.history.v1.HistoryEvent.nexus_operation_scheduled_event_attributes:type_name -> temporal.api.history.v1.NexusOperationScheduledEventAttributes + 48, // 191: temporal.api.history.v1.HistoryEvent.nexus_operation_started_event_attributes:type_name -> temporal.api.history.v1.NexusOperationStartedEventAttributes + 49, // 192: temporal.api.history.v1.HistoryEvent.nexus_operation_completed_event_attributes:type_name -> temporal.api.history.v1.NexusOperationCompletedEventAttributes + 50, // 193: temporal.api.history.v1.HistoryEvent.nexus_operation_failed_event_attributes:type_name -> temporal.api.history.v1.NexusOperationFailedEventAttributes + 52, // 194: temporal.api.history.v1.HistoryEvent.nexus_operation_canceled_event_attributes:type_name -> temporal.api.history.v1.NexusOperationCanceledEventAttributes + 51, // 195: temporal.api.history.v1.HistoryEvent.nexus_operation_timed_out_event_attributes:type_name -> temporal.api.history.v1.NexusOperationTimedOutEventAttributes + 53, // 196: temporal.api.history.v1.HistoryEvent.nexus_operation_cancel_requested_event_attributes:type_name -> temporal.api.history.v1.NexusOperationCancelRequestedEventAttributes + 54, // 197: temporal.api.history.v1.History.events:type_name -> temporal.api.history.v1.HistoryEvent + 61, // 198: temporal.api.history.v1.MarkerRecordedEventAttributes.DetailsEntry.value:type_name -> temporal.api.common.v1.Payloads + 199, // [199:199] is the sub-list for method output_type + 199, // [199:199] is the sub-list for method input_type + 199, // [199:199] is the sub-list for extension type_name + 199, // [199:199] is the sub-list for extension extendee + 0, // [0:199] is the sub-list for field type_name +} + +func init() { file_temporal_api_history_v1_message_proto_init() } +func file_temporal_api_history_v1_message_proto_init() { + if File_temporal_api_history_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_history_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionStartedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionCompletedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionFailedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionTimedOutEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionContinuedAsNewEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowTaskScheduledEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowTaskStartedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowTaskCompletedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowTaskTimedOutEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowTaskFailedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*ActivityTaskScheduledEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[11].Exporter = func(v any, i int) any { + switch v := v.(*ActivityTaskStartedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[12].Exporter = func(v any, i int) any { + switch v := v.(*ActivityTaskCompletedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[13].Exporter = func(v any, i int) any { + switch v := v.(*ActivityTaskFailedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[14].Exporter = func(v any, i int) any { + switch v := v.(*ActivityTaskTimedOutEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[15].Exporter = func(v any, i int) any { + switch v := v.(*ActivityTaskCancelRequestedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[16].Exporter = func(v any, i int) any { + switch v := v.(*ActivityTaskCanceledEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[17].Exporter = func(v any, i int) any { + switch v := v.(*TimerStartedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[18].Exporter = func(v any, i int) any { + switch v := v.(*TimerFiredEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[19].Exporter = func(v any, i int) any { + switch v := v.(*TimerCanceledEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[20].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionCancelRequestedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[21].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionCanceledEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[22].Exporter = func(v any, i int) any { + switch v := v.(*MarkerRecordedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[23].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionSignaledEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[24].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionTerminatedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[25].Exporter = func(v any, i int) any { + switch v := v.(*RequestCancelExternalWorkflowExecutionInitiatedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[26].Exporter = func(v any, i int) any { + switch v := v.(*RequestCancelExternalWorkflowExecutionFailedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[27].Exporter = func(v any, i int) any { + switch v := v.(*ExternalWorkflowExecutionCancelRequestedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[28].Exporter = func(v any, i int) any { + switch v := v.(*SignalExternalWorkflowExecutionInitiatedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[29].Exporter = func(v any, i int) any { + switch v := v.(*SignalExternalWorkflowExecutionFailedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[30].Exporter = func(v any, i int) any { + switch v := v.(*ExternalWorkflowExecutionSignaledEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[31].Exporter = func(v any, i int) any { + switch v := v.(*UpsertWorkflowSearchAttributesEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[32].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowPropertiesModifiedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[33].Exporter = func(v any, i int) any { + switch v := v.(*StartChildWorkflowExecutionInitiatedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[34].Exporter = func(v any, i int) any { + switch v := v.(*StartChildWorkflowExecutionFailedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[35].Exporter = func(v any, i int) any { + switch v := v.(*ChildWorkflowExecutionStartedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[36].Exporter = func(v any, i int) any { + switch v := v.(*ChildWorkflowExecutionCompletedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[37].Exporter = func(v any, i int) any { + switch v := v.(*ChildWorkflowExecutionFailedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[38].Exporter = func(v any, i int) any { + switch v := v.(*ChildWorkflowExecutionCanceledEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[39].Exporter = func(v any, i int) any { + switch v := v.(*ChildWorkflowExecutionTimedOutEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[40].Exporter = func(v any, i int) any { + switch v := v.(*ChildWorkflowExecutionTerminatedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[41].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowPropertiesModifiedExternallyEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[42].Exporter = func(v any, i int) any { + switch v := v.(*ActivityPropertiesModifiedExternallyEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[43].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionUpdateAcceptedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[44].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionUpdateCompletedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[45].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionUpdateRejectedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[46].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionUpdateAdmittedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[47].Exporter = func(v any, i int) any { + switch v := v.(*NexusOperationScheduledEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[48].Exporter = func(v any, i int) any { + switch v := v.(*NexusOperationStartedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[49].Exporter = func(v any, i int) any { + switch v := v.(*NexusOperationCompletedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[50].Exporter = func(v any, i int) any { + switch v := v.(*NexusOperationFailedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[51].Exporter = func(v any, i int) any { + switch v := v.(*NexusOperationTimedOutEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[52].Exporter = func(v any, i int) any { + switch v := v.(*NexusOperationCanceledEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[53].Exporter = func(v any, i int) any { + switch v := v.(*NexusOperationCancelRequestedEventAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[54].Exporter = func(v any, i int) any { + switch v := v.(*HistoryEvent); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_history_v1_message_proto_msgTypes[55].Exporter = func(v any, i int) any { + switch v := v.(*History); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_temporal_api_history_v1_message_proto_msgTypes[54].OneofWrappers = []any{ + (*HistoryEvent_WorkflowExecutionStartedEventAttributes)(nil), + (*HistoryEvent_WorkflowExecutionCompletedEventAttributes)(nil), + (*HistoryEvent_WorkflowExecutionFailedEventAttributes)(nil), + (*HistoryEvent_WorkflowExecutionTimedOutEventAttributes)(nil), + (*HistoryEvent_WorkflowTaskScheduledEventAttributes)(nil), + (*HistoryEvent_WorkflowTaskStartedEventAttributes)(nil), + (*HistoryEvent_WorkflowTaskCompletedEventAttributes)(nil), + (*HistoryEvent_WorkflowTaskTimedOutEventAttributes)(nil), + (*HistoryEvent_WorkflowTaskFailedEventAttributes)(nil), + (*HistoryEvent_ActivityTaskScheduledEventAttributes)(nil), + (*HistoryEvent_ActivityTaskStartedEventAttributes)(nil), + (*HistoryEvent_ActivityTaskCompletedEventAttributes)(nil), + (*HistoryEvent_ActivityTaskFailedEventAttributes)(nil), + (*HistoryEvent_ActivityTaskTimedOutEventAttributes)(nil), + (*HistoryEvent_TimerStartedEventAttributes)(nil), + (*HistoryEvent_TimerFiredEventAttributes)(nil), + (*HistoryEvent_ActivityTaskCancelRequestedEventAttributes)(nil), + (*HistoryEvent_ActivityTaskCanceledEventAttributes)(nil), + (*HistoryEvent_TimerCanceledEventAttributes)(nil), + (*HistoryEvent_MarkerRecordedEventAttributes)(nil), + (*HistoryEvent_WorkflowExecutionSignaledEventAttributes)(nil), + (*HistoryEvent_WorkflowExecutionTerminatedEventAttributes)(nil), + (*HistoryEvent_WorkflowExecutionCancelRequestedEventAttributes)(nil), + (*HistoryEvent_WorkflowExecutionCanceledEventAttributes)(nil), + (*HistoryEvent_RequestCancelExternalWorkflowExecutionInitiatedEventAttributes)(nil), + (*HistoryEvent_RequestCancelExternalWorkflowExecutionFailedEventAttributes)(nil), + (*HistoryEvent_ExternalWorkflowExecutionCancelRequestedEventAttributes)(nil), + (*HistoryEvent_WorkflowExecutionContinuedAsNewEventAttributes)(nil), + (*HistoryEvent_StartChildWorkflowExecutionInitiatedEventAttributes)(nil), + (*HistoryEvent_StartChildWorkflowExecutionFailedEventAttributes)(nil), + (*HistoryEvent_ChildWorkflowExecutionStartedEventAttributes)(nil), + (*HistoryEvent_ChildWorkflowExecutionCompletedEventAttributes)(nil), + (*HistoryEvent_ChildWorkflowExecutionFailedEventAttributes)(nil), + (*HistoryEvent_ChildWorkflowExecutionCanceledEventAttributes)(nil), + (*HistoryEvent_ChildWorkflowExecutionTimedOutEventAttributes)(nil), + (*HistoryEvent_ChildWorkflowExecutionTerminatedEventAttributes)(nil), + (*HistoryEvent_SignalExternalWorkflowExecutionInitiatedEventAttributes)(nil), + (*HistoryEvent_SignalExternalWorkflowExecutionFailedEventAttributes)(nil), + (*HistoryEvent_ExternalWorkflowExecutionSignaledEventAttributes)(nil), + (*HistoryEvent_UpsertWorkflowSearchAttributesEventAttributes)(nil), + (*HistoryEvent_WorkflowExecutionUpdateAcceptedEventAttributes)(nil), + (*HistoryEvent_WorkflowExecutionUpdateRejectedEventAttributes)(nil), + (*HistoryEvent_WorkflowExecutionUpdateCompletedEventAttributes)(nil), + (*HistoryEvent_WorkflowPropertiesModifiedExternallyEventAttributes)(nil), + (*HistoryEvent_ActivityPropertiesModifiedExternallyEventAttributes)(nil), + (*HistoryEvent_WorkflowPropertiesModifiedEventAttributes)(nil), + (*HistoryEvent_WorkflowExecutionUpdateAdmittedEventAttributes)(nil), + (*HistoryEvent_NexusOperationScheduledEventAttributes)(nil), + (*HistoryEvent_NexusOperationStartedEventAttributes)(nil), + (*HistoryEvent_NexusOperationCompletedEventAttributes)(nil), + (*HistoryEvent_NexusOperationFailedEventAttributes)(nil), + (*HistoryEvent_NexusOperationCanceledEventAttributes)(nil), + (*HistoryEvent_NexusOperationTimedOutEventAttributes)(nil), + (*HistoryEvent_NexusOperationCancelRequestedEventAttributes)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_history_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 58, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_history_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_history_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_history_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_history_v1_message_proto = out.File + file_temporal_api_history_v1_message_proto_rawDesc = nil + file_temporal_api_history_v1_message_proto_goTypes = nil + file_temporal_api_history_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/internal/protojson/LICENSE b/vendor/go.temporal.io/api/internal/protojson/LICENSE new file mode 100644 index 00000000000..49ea0f92882 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/LICENSE @@ -0,0 +1,27 @@ +Copyright (c) 2018 The Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/go.temporal.io/api/internal/protojson/README.md b/vendor/go.temporal.io/api/internal/protojson/README.md new file mode 100644 index 00000000000..29d62a9d533 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/README.md @@ -0,0 +1,9 @@ +_Here lies a vendored copy of protojson with some minor alterations._ + +Unfortunately we need a few things that the official stack won't let us do: + +1. We need to support our old JSON with `camelCase` enums while we migrate to the canonical `SCREAMING_SNAKE_CASE` enums +2. We've decided to support a [shorthand](https://github.com/temporalio/proposals/blob/master/api/http-api.md#payload-formatting) JSON serialization for our Payload type so that our users have a better experience + + +All code herein is governed by the LICENSE file in this directory (barring the `maybe_marshal` code we added). diff --git a/vendor/go.temporal.io/api/internal/protojson/errors/errors.go b/vendor/go.temporal.io/api/internal/protojson/errors/errors.go new file mode 100644 index 00000000000..3d2ff800f4b --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/errors/errors.go @@ -0,0 +1,79 @@ +// Copyright 2018 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package errors implements functions to manipulate errors. +package errors + +import ( + "errors" + "fmt" +) + +// Error is a sentinel matching all errors produced by this package. +var Error = errors.New("temporalprotobuf error") + +// New formats a string according to the format specifier and arguments and +// returns an error that has a "temporalproto" prefix. +func New(f string, x ...interface{}) error { + return &prefixError{s: format(f, x...)} +} + +type prefixError struct{ s string } + +const prefix = "temporalproto: " + +func (e *prefixError) Error() string { + return prefix + e.s +} + +func (e *prefixError) Unwrap() error { + return Error +} + +// Wrap returns an error that has a "proto" prefix, the formatted string described +// by the format specifier and arguments, and a suffix of err. The error wraps err. +func Wrap(err error, f string, x ...interface{}) error { + return &wrapError{ + s: format(f, x...), + err: err, + } +} + +type wrapError struct { + s string + err error +} + +func (e *wrapError) Error() string { + return format("%v%v: %v", prefix, e.s, e.err) +} + +func (e *wrapError) Unwrap() error { + return e.err +} + +func (e *wrapError) Is(target error) bool { + return target == Error +} + +func format(f string, x ...interface{}) string { + // avoid prefix when chaining + for i := 0; i < len(x); i++ { + switch e := x[i].(type) { + case *prefixError: + x[i] = e.s + case *wrapError: + x[i] = format("%v: %v", e.s, e.err) + } + } + return fmt.Sprintf(f, x...) +} + +func InvalidUTF8(name string) error { + return New("field %v contains invalid UTF-8", name) +} + +func RequiredNotSet(name string) error { + return New("required field %v not set", name) +} diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/any_gen.go b/vendor/go.temporal.io/api/internal/protojson/genid/any_gen.go new file mode 100644 index 00000000000..e6f7d47ab6d --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/any_gen.go @@ -0,0 +1,34 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Code generated by generate-protos. DO NOT EDIT. + +package genid + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" +) + +const File_google_protobuf_any_proto = "google/protobuf/any.proto" + +// Names for google.protobuf.Any. +const ( + Any_message_name protoreflect.Name = "Any" + Any_message_fullname protoreflect.FullName = "google.protobuf.Any" +) + +// Field names for google.protobuf.Any. +const ( + Any_TypeUrl_field_name protoreflect.Name = "type_url" + Any_Value_field_name protoreflect.Name = "value" + + Any_TypeUrl_field_fullname protoreflect.FullName = "google.protobuf.Any.type_url" + Any_Value_field_fullname protoreflect.FullName = "google.protobuf.Any.value" +) + +// Field numbers for google.protobuf.Any. +const ( + Any_TypeUrl_field_number protoreflect.FieldNumber = 1 + Any_Value_field_number protoreflect.FieldNumber = 2 +) diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/api_gen.go b/vendor/go.temporal.io/api/internal/protojson/genid/api_gen.go new file mode 100644 index 00000000000..df8f9185013 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/api_gen.go @@ -0,0 +1,106 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Code generated by generate-protos. DO NOT EDIT. + +package genid + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" +) + +const File_google_protobuf_api_proto = "google/protobuf/api.proto" + +// Names for google.protobuf.Api. +const ( + Api_message_name protoreflect.Name = "Api" + Api_message_fullname protoreflect.FullName = "google.protobuf.Api" +) + +// Field names for google.protobuf.Api. +const ( + Api_Name_field_name protoreflect.Name = "name" + Api_Methods_field_name protoreflect.Name = "methods" + Api_Options_field_name protoreflect.Name = "options" + Api_Version_field_name protoreflect.Name = "version" + Api_SourceContext_field_name protoreflect.Name = "source_context" + Api_Mixins_field_name protoreflect.Name = "mixins" + Api_Syntax_field_name protoreflect.Name = "syntax" + + Api_Name_field_fullname protoreflect.FullName = "google.protobuf.Api.name" + Api_Methods_field_fullname protoreflect.FullName = "google.protobuf.Api.methods" + Api_Options_field_fullname protoreflect.FullName = "google.protobuf.Api.options" + Api_Version_field_fullname protoreflect.FullName = "google.protobuf.Api.version" + Api_SourceContext_field_fullname protoreflect.FullName = "google.protobuf.Api.source_context" + Api_Mixins_field_fullname protoreflect.FullName = "google.protobuf.Api.mixins" + Api_Syntax_field_fullname protoreflect.FullName = "google.protobuf.Api.syntax" +) + +// Field numbers for google.protobuf.Api. +const ( + Api_Name_field_number protoreflect.FieldNumber = 1 + Api_Methods_field_number protoreflect.FieldNumber = 2 + Api_Options_field_number protoreflect.FieldNumber = 3 + Api_Version_field_number protoreflect.FieldNumber = 4 + Api_SourceContext_field_number protoreflect.FieldNumber = 5 + Api_Mixins_field_number protoreflect.FieldNumber = 6 + Api_Syntax_field_number protoreflect.FieldNumber = 7 +) + +// Names for google.protobuf.Method. +const ( + Method_message_name protoreflect.Name = "Method" + Method_message_fullname protoreflect.FullName = "google.protobuf.Method" +) + +// Field names for google.protobuf.Method. +const ( + Method_Name_field_name protoreflect.Name = "name" + Method_RequestTypeUrl_field_name protoreflect.Name = "request_type_url" + Method_RequestStreaming_field_name protoreflect.Name = "request_streaming" + Method_ResponseTypeUrl_field_name protoreflect.Name = "response_type_url" + Method_ResponseStreaming_field_name protoreflect.Name = "response_streaming" + Method_Options_field_name protoreflect.Name = "options" + Method_Syntax_field_name protoreflect.Name = "syntax" + + Method_Name_field_fullname protoreflect.FullName = "google.protobuf.Method.name" + Method_RequestTypeUrl_field_fullname protoreflect.FullName = "google.protobuf.Method.request_type_url" + Method_RequestStreaming_field_fullname protoreflect.FullName = "google.protobuf.Method.request_streaming" + Method_ResponseTypeUrl_field_fullname protoreflect.FullName = "google.protobuf.Method.response_type_url" + Method_ResponseStreaming_field_fullname protoreflect.FullName = "google.protobuf.Method.response_streaming" + Method_Options_field_fullname protoreflect.FullName = "google.protobuf.Method.options" + Method_Syntax_field_fullname protoreflect.FullName = "google.protobuf.Method.syntax" +) + +// Field numbers for google.protobuf.Method. +const ( + Method_Name_field_number protoreflect.FieldNumber = 1 + Method_RequestTypeUrl_field_number protoreflect.FieldNumber = 2 + Method_RequestStreaming_field_number protoreflect.FieldNumber = 3 + Method_ResponseTypeUrl_field_number protoreflect.FieldNumber = 4 + Method_ResponseStreaming_field_number protoreflect.FieldNumber = 5 + Method_Options_field_number protoreflect.FieldNumber = 6 + Method_Syntax_field_number protoreflect.FieldNumber = 7 +) + +// Names for google.protobuf.Mixin. +const ( + Mixin_message_name protoreflect.Name = "Mixin" + Mixin_message_fullname protoreflect.FullName = "google.protobuf.Mixin" +) + +// Field names for google.protobuf.Mixin. +const ( + Mixin_Name_field_name protoreflect.Name = "name" + Mixin_Root_field_name protoreflect.Name = "root" + + Mixin_Name_field_fullname protoreflect.FullName = "google.protobuf.Mixin.name" + Mixin_Root_field_fullname protoreflect.FullName = "google.protobuf.Mixin.root" +) + +// Field numbers for google.protobuf.Mixin. +const ( + Mixin_Name_field_number protoreflect.FieldNumber = 1 + Mixin_Root_field_number protoreflect.FieldNumber = 2 +) diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/descriptor_gen.go b/vendor/go.temporal.io/api/internal/protojson/genid/descriptor_gen.go new file mode 100644 index 00000000000..8f94230ea1c --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/descriptor_gen.go @@ -0,0 +1,1087 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Code generated by generate-protos. DO NOT EDIT. + +package genid + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" +) + +const File_google_protobuf_descriptor_proto = "google/protobuf/descriptor.proto" + +// Full and short names for google.protobuf.Edition. +const ( + Edition_enum_fullname = "google.protobuf.Edition" + Edition_enum_name = "Edition" +) + +// Names for google.protobuf.FileDescriptorSet. +const ( + FileDescriptorSet_message_name protoreflect.Name = "FileDescriptorSet" + FileDescriptorSet_message_fullname protoreflect.FullName = "google.protobuf.FileDescriptorSet" +) + +// Field names for google.protobuf.FileDescriptorSet. +const ( + FileDescriptorSet_File_field_name protoreflect.Name = "file" + + FileDescriptorSet_File_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorSet.file" +) + +// Field numbers for google.protobuf.FileDescriptorSet. +const ( + FileDescriptorSet_File_field_number protoreflect.FieldNumber = 1 +) + +// Names for google.protobuf.FileDescriptorProto. +const ( + FileDescriptorProto_message_name protoreflect.Name = "FileDescriptorProto" + FileDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto" +) + +// Field names for google.protobuf.FileDescriptorProto. +const ( + FileDescriptorProto_Name_field_name protoreflect.Name = "name" + FileDescriptorProto_Package_field_name protoreflect.Name = "package" + FileDescriptorProto_Dependency_field_name protoreflect.Name = "dependency" + FileDescriptorProto_PublicDependency_field_name protoreflect.Name = "public_dependency" + FileDescriptorProto_WeakDependency_field_name protoreflect.Name = "weak_dependency" + FileDescriptorProto_MessageType_field_name protoreflect.Name = "message_type" + FileDescriptorProto_EnumType_field_name protoreflect.Name = "enum_type" + FileDescriptorProto_Service_field_name protoreflect.Name = "service" + FileDescriptorProto_Extension_field_name protoreflect.Name = "extension" + FileDescriptorProto_Options_field_name protoreflect.Name = "options" + FileDescriptorProto_SourceCodeInfo_field_name protoreflect.Name = "source_code_info" + FileDescriptorProto_Syntax_field_name protoreflect.Name = "syntax" + FileDescriptorProto_Edition_field_name protoreflect.Name = "edition" + + FileDescriptorProto_Name_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.name" + FileDescriptorProto_Package_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.package" + FileDescriptorProto_Dependency_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.dependency" + FileDescriptorProto_PublicDependency_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.public_dependency" + FileDescriptorProto_WeakDependency_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.weak_dependency" + FileDescriptorProto_MessageType_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.message_type" + FileDescriptorProto_EnumType_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.enum_type" + FileDescriptorProto_Service_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.service" + FileDescriptorProto_Extension_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.extension" + FileDescriptorProto_Options_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.options" + FileDescriptorProto_SourceCodeInfo_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.source_code_info" + FileDescriptorProto_Syntax_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.syntax" + FileDescriptorProto_Edition_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.edition" +) + +// Field numbers for google.protobuf.FileDescriptorProto. +const ( + FileDescriptorProto_Name_field_number protoreflect.FieldNumber = 1 + FileDescriptorProto_Package_field_number protoreflect.FieldNumber = 2 + FileDescriptorProto_Dependency_field_number protoreflect.FieldNumber = 3 + FileDescriptorProto_PublicDependency_field_number protoreflect.FieldNumber = 10 + FileDescriptorProto_WeakDependency_field_number protoreflect.FieldNumber = 11 + FileDescriptorProto_MessageType_field_number protoreflect.FieldNumber = 4 + FileDescriptorProto_EnumType_field_number protoreflect.FieldNumber = 5 + FileDescriptorProto_Service_field_number protoreflect.FieldNumber = 6 + FileDescriptorProto_Extension_field_number protoreflect.FieldNumber = 7 + FileDescriptorProto_Options_field_number protoreflect.FieldNumber = 8 + FileDescriptorProto_SourceCodeInfo_field_number protoreflect.FieldNumber = 9 + FileDescriptorProto_Syntax_field_number protoreflect.FieldNumber = 12 + FileDescriptorProto_Edition_field_number protoreflect.FieldNumber = 14 +) + +// Names for google.protobuf.DescriptorProto. +const ( + DescriptorProto_message_name protoreflect.Name = "DescriptorProto" + DescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.DescriptorProto" +) + +// Field names for google.protobuf.DescriptorProto. +const ( + DescriptorProto_Name_field_name protoreflect.Name = "name" + DescriptorProto_Field_field_name protoreflect.Name = "field" + DescriptorProto_Extension_field_name protoreflect.Name = "extension" + DescriptorProto_NestedType_field_name protoreflect.Name = "nested_type" + DescriptorProto_EnumType_field_name protoreflect.Name = "enum_type" + DescriptorProto_ExtensionRange_field_name protoreflect.Name = "extension_range" + DescriptorProto_OneofDecl_field_name protoreflect.Name = "oneof_decl" + DescriptorProto_Options_field_name protoreflect.Name = "options" + DescriptorProto_ReservedRange_field_name protoreflect.Name = "reserved_range" + DescriptorProto_ReservedName_field_name protoreflect.Name = "reserved_name" + + DescriptorProto_Name_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.name" + DescriptorProto_Field_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.field" + DescriptorProto_Extension_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.extension" + DescriptorProto_NestedType_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.nested_type" + DescriptorProto_EnumType_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.enum_type" + DescriptorProto_ExtensionRange_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.extension_range" + DescriptorProto_OneofDecl_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.oneof_decl" + DescriptorProto_Options_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.options" + DescriptorProto_ReservedRange_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.reserved_range" + DescriptorProto_ReservedName_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.reserved_name" +) + +// Field numbers for google.protobuf.DescriptorProto. +const ( + DescriptorProto_Name_field_number protoreflect.FieldNumber = 1 + DescriptorProto_Field_field_number protoreflect.FieldNumber = 2 + DescriptorProto_Extension_field_number protoreflect.FieldNumber = 6 + DescriptorProto_NestedType_field_number protoreflect.FieldNumber = 3 + DescriptorProto_EnumType_field_number protoreflect.FieldNumber = 4 + DescriptorProto_ExtensionRange_field_number protoreflect.FieldNumber = 5 + DescriptorProto_OneofDecl_field_number protoreflect.FieldNumber = 8 + DescriptorProto_Options_field_number protoreflect.FieldNumber = 7 + DescriptorProto_ReservedRange_field_number protoreflect.FieldNumber = 9 + DescriptorProto_ReservedName_field_number protoreflect.FieldNumber = 10 +) + +// Names for google.protobuf.DescriptorProto.ExtensionRange. +const ( + DescriptorProto_ExtensionRange_message_name protoreflect.Name = "ExtensionRange" + DescriptorProto_ExtensionRange_message_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.ExtensionRange" +) + +// Field names for google.protobuf.DescriptorProto.ExtensionRange. +const ( + DescriptorProto_ExtensionRange_Start_field_name protoreflect.Name = "start" + DescriptorProto_ExtensionRange_End_field_name protoreflect.Name = "end" + DescriptorProto_ExtensionRange_Options_field_name protoreflect.Name = "options" + + DescriptorProto_ExtensionRange_Start_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.ExtensionRange.start" + DescriptorProto_ExtensionRange_End_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.ExtensionRange.end" + DescriptorProto_ExtensionRange_Options_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.ExtensionRange.options" +) + +// Field numbers for google.protobuf.DescriptorProto.ExtensionRange. +const ( + DescriptorProto_ExtensionRange_Start_field_number protoreflect.FieldNumber = 1 + DescriptorProto_ExtensionRange_End_field_number protoreflect.FieldNumber = 2 + DescriptorProto_ExtensionRange_Options_field_number protoreflect.FieldNumber = 3 +) + +// Names for google.protobuf.DescriptorProto.ReservedRange. +const ( + DescriptorProto_ReservedRange_message_name protoreflect.Name = "ReservedRange" + DescriptorProto_ReservedRange_message_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.ReservedRange" +) + +// Field names for google.protobuf.DescriptorProto.ReservedRange. +const ( + DescriptorProto_ReservedRange_Start_field_name protoreflect.Name = "start" + DescriptorProto_ReservedRange_End_field_name protoreflect.Name = "end" + + DescriptorProto_ReservedRange_Start_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.ReservedRange.start" + DescriptorProto_ReservedRange_End_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.ReservedRange.end" +) + +// Field numbers for google.protobuf.DescriptorProto.ReservedRange. +const ( + DescriptorProto_ReservedRange_Start_field_number protoreflect.FieldNumber = 1 + DescriptorProto_ReservedRange_End_field_number protoreflect.FieldNumber = 2 +) + +// Names for google.protobuf.ExtensionRangeOptions. +const ( + ExtensionRangeOptions_message_name protoreflect.Name = "ExtensionRangeOptions" + ExtensionRangeOptions_message_fullname protoreflect.FullName = "google.protobuf.ExtensionRangeOptions" +) + +// Field names for google.protobuf.ExtensionRangeOptions. +const ( + ExtensionRangeOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option" + ExtensionRangeOptions_Declaration_field_name protoreflect.Name = "declaration" + ExtensionRangeOptions_Features_field_name protoreflect.Name = "features" + ExtensionRangeOptions_Verification_field_name protoreflect.Name = "verification" + + ExtensionRangeOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.ExtensionRangeOptions.uninterpreted_option" + ExtensionRangeOptions_Declaration_field_fullname protoreflect.FullName = "google.protobuf.ExtensionRangeOptions.declaration" + ExtensionRangeOptions_Features_field_fullname protoreflect.FullName = "google.protobuf.ExtensionRangeOptions.features" + ExtensionRangeOptions_Verification_field_fullname protoreflect.FullName = "google.protobuf.ExtensionRangeOptions.verification" +) + +// Field numbers for google.protobuf.ExtensionRangeOptions. +const ( + ExtensionRangeOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999 + ExtensionRangeOptions_Declaration_field_number protoreflect.FieldNumber = 2 + ExtensionRangeOptions_Features_field_number protoreflect.FieldNumber = 50 + ExtensionRangeOptions_Verification_field_number protoreflect.FieldNumber = 3 +) + +// Full and short names for google.protobuf.ExtensionRangeOptions.VerificationState. +const ( + ExtensionRangeOptions_VerificationState_enum_fullname = "google.protobuf.ExtensionRangeOptions.VerificationState" + ExtensionRangeOptions_VerificationState_enum_name = "VerificationState" +) + +// Names for google.protobuf.ExtensionRangeOptions.Declaration. +const ( + ExtensionRangeOptions_Declaration_message_name protoreflect.Name = "Declaration" + ExtensionRangeOptions_Declaration_message_fullname protoreflect.FullName = "google.protobuf.ExtensionRangeOptions.Declaration" +) + +// Field names for google.protobuf.ExtensionRangeOptions.Declaration. +const ( + ExtensionRangeOptions_Declaration_Number_field_name protoreflect.Name = "number" + ExtensionRangeOptions_Declaration_FullName_field_name protoreflect.Name = "full_name" + ExtensionRangeOptions_Declaration_Type_field_name protoreflect.Name = "type" + ExtensionRangeOptions_Declaration_Reserved_field_name protoreflect.Name = "reserved" + ExtensionRangeOptions_Declaration_Repeated_field_name protoreflect.Name = "repeated" + + ExtensionRangeOptions_Declaration_Number_field_fullname protoreflect.FullName = "google.protobuf.ExtensionRangeOptions.Declaration.number" + ExtensionRangeOptions_Declaration_FullName_field_fullname protoreflect.FullName = "google.protobuf.ExtensionRangeOptions.Declaration.full_name" + ExtensionRangeOptions_Declaration_Type_field_fullname protoreflect.FullName = "google.protobuf.ExtensionRangeOptions.Declaration.type" + ExtensionRangeOptions_Declaration_Reserved_field_fullname protoreflect.FullName = "google.protobuf.ExtensionRangeOptions.Declaration.reserved" + ExtensionRangeOptions_Declaration_Repeated_field_fullname protoreflect.FullName = "google.protobuf.ExtensionRangeOptions.Declaration.repeated" +) + +// Field numbers for google.protobuf.ExtensionRangeOptions.Declaration. +const ( + ExtensionRangeOptions_Declaration_Number_field_number protoreflect.FieldNumber = 1 + ExtensionRangeOptions_Declaration_FullName_field_number protoreflect.FieldNumber = 2 + ExtensionRangeOptions_Declaration_Type_field_number protoreflect.FieldNumber = 3 + ExtensionRangeOptions_Declaration_Reserved_field_number protoreflect.FieldNumber = 5 + ExtensionRangeOptions_Declaration_Repeated_field_number protoreflect.FieldNumber = 6 +) + +// Names for google.protobuf.FieldDescriptorProto. +const ( + FieldDescriptorProto_message_name protoreflect.Name = "FieldDescriptorProto" + FieldDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto" +) + +// Field names for google.protobuf.FieldDescriptorProto. +const ( + FieldDescriptorProto_Name_field_name protoreflect.Name = "name" + FieldDescriptorProto_Number_field_name protoreflect.Name = "number" + FieldDescriptorProto_Label_field_name protoreflect.Name = "label" + FieldDescriptorProto_Type_field_name protoreflect.Name = "type" + FieldDescriptorProto_TypeName_field_name protoreflect.Name = "type_name" + FieldDescriptorProto_Extendee_field_name protoreflect.Name = "extendee" + FieldDescriptorProto_DefaultValue_field_name protoreflect.Name = "default_value" + FieldDescriptorProto_OneofIndex_field_name protoreflect.Name = "oneof_index" + FieldDescriptorProto_JsonName_field_name protoreflect.Name = "json_name" + FieldDescriptorProto_Options_field_name protoreflect.Name = "options" + FieldDescriptorProto_Proto3Optional_field_name protoreflect.Name = "proto3_optional" + + FieldDescriptorProto_Name_field_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto.name" + FieldDescriptorProto_Number_field_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto.number" + FieldDescriptorProto_Label_field_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto.label" + FieldDescriptorProto_Type_field_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto.type" + FieldDescriptorProto_TypeName_field_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto.type_name" + FieldDescriptorProto_Extendee_field_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto.extendee" + FieldDescriptorProto_DefaultValue_field_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto.default_value" + FieldDescriptorProto_OneofIndex_field_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto.oneof_index" + FieldDescriptorProto_JsonName_field_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto.json_name" + FieldDescriptorProto_Options_field_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto.options" + FieldDescriptorProto_Proto3Optional_field_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto.proto3_optional" +) + +// Field numbers for google.protobuf.FieldDescriptorProto. +const ( + FieldDescriptorProto_Name_field_number protoreflect.FieldNumber = 1 + FieldDescriptorProto_Number_field_number protoreflect.FieldNumber = 3 + FieldDescriptorProto_Label_field_number protoreflect.FieldNumber = 4 + FieldDescriptorProto_Type_field_number protoreflect.FieldNumber = 5 + FieldDescriptorProto_TypeName_field_number protoreflect.FieldNumber = 6 + FieldDescriptorProto_Extendee_field_number protoreflect.FieldNumber = 2 + FieldDescriptorProto_DefaultValue_field_number protoreflect.FieldNumber = 7 + FieldDescriptorProto_OneofIndex_field_number protoreflect.FieldNumber = 9 + FieldDescriptorProto_JsonName_field_number protoreflect.FieldNumber = 10 + FieldDescriptorProto_Options_field_number protoreflect.FieldNumber = 8 + FieldDescriptorProto_Proto3Optional_field_number protoreflect.FieldNumber = 17 +) + +// Full and short names for google.protobuf.FieldDescriptorProto.Type. +const ( + FieldDescriptorProto_Type_enum_fullname = "google.protobuf.FieldDescriptorProto.Type" + FieldDescriptorProto_Type_enum_name = "Type" +) + +// Full and short names for google.protobuf.FieldDescriptorProto.Label. +const ( + FieldDescriptorProto_Label_enum_fullname = "google.protobuf.FieldDescriptorProto.Label" + FieldDescriptorProto_Label_enum_name = "Label" +) + +// Names for google.protobuf.OneofDescriptorProto. +const ( + OneofDescriptorProto_message_name protoreflect.Name = "OneofDescriptorProto" + OneofDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.OneofDescriptorProto" +) + +// Field names for google.protobuf.OneofDescriptorProto. +const ( + OneofDescriptorProto_Name_field_name protoreflect.Name = "name" + OneofDescriptorProto_Options_field_name protoreflect.Name = "options" + + OneofDescriptorProto_Name_field_fullname protoreflect.FullName = "google.protobuf.OneofDescriptorProto.name" + OneofDescriptorProto_Options_field_fullname protoreflect.FullName = "google.protobuf.OneofDescriptorProto.options" +) + +// Field numbers for google.protobuf.OneofDescriptorProto. +const ( + OneofDescriptorProto_Name_field_number protoreflect.FieldNumber = 1 + OneofDescriptorProto_Options_field_number protoreflect.FieldNumber = 2 +) + +// Names for google.protobuf.EnumDescriptorProto. +const ( + EnumDescriptorProto_message_name protoreflect.Name = "EnumDescriptorProto" + EnumDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.EnumDescriptorProto" +) + +// Field names for google.protobuf.EnumDescriptorProto. +const ( + EnumDescriptorProto_Name_field_name protoreflect.Name = "name" + EnumDescriptorProto_Value_field_name protoreflect.Name = "value" + EnumDescriptorProto_Options_field_name protoreflect.Name = "options" + EnumDescriptorProto_ReservedRange_field_name protoreflect.Name = "reserved_range" + EnumDescriptorProto_ReservedName_field_name protoreflect.Name = "reserved_name" + + EnumDescriptorProto_Name_field_fullname protoreflect.FullName = "google.protobuf.EnumDescriptorProto.name" + EnumDescriptorProto_Value_field_fullname protoreflect.FullName = "google.protobuf.EnumDescriptorProto.value" + EnumDescriptorProto_Options_field_fullname protoreflect.FullName = "google.protobuf.EnumDescriptorProto.options" + EnumDescriptorProto_ReservedRange_field_fullname protoreflect.FullName = "google.protobuf.EnumDescriptorProto.reserved_range" + EnumDescriptorProto_ReservedName_field_fullname protoreflect.FullName = "google.protobuf.EnumDescriptorProto.reserved_name" +) + +// Field numbers for google.protobuf.EnumDescriptorProto. +const ( + EnumDescriptorProto_Name_field_number protoreflect.FieldNumber = 1 + EnumDescriptorProto_Value_field_number protoreflect.FieldNumber = 2 + EnumDescriptorProto_Options_field_number protoreflect.FieldNumber = 3 + EnumDescriptorProto_ReservedRange_field_number protoreflect.FieldNumber = 4 + EnumDescriptorProto_ReservedName_field_number protoreflect.FieldNumber = 5 +) + +// Names for google.protobuf.EnumDescriptorProto.EnumReservedRange. +const ( + EnumDescriptorProto_EnumReservedRange_message_name protoreflect.Name = "EnumReservedRange" + EnumDescriptorProto_EnumReservedRange_message_fullname protoreflect.FullName = "google.protobuf.EnumDescriptorProto.EnumReservedRange" +) + +// Field names for google.protobuf.EnumDescriptorProto.EnumReservedRange. +const ( + EnumDescriptorProto_EnumReservedRange_Start_field_name protoreflect.Name = "start" + EnumDescriptorProto_EnumReservedRange_End_field_name protoreflect.Name = "end" + + EnumDescriptorProto_EnumReservedRange_Start_field_fullname protoreflect.FullName = "google.protobuf.EnumDescriptorProto.EnumReservedRange.start" + EnumDescriptorProto_EnumReservedRange_End_field_fullname protoreflect.FullName = "google.protobuf.EnumDescriptorProto.EnumReservedRange.end" +) + +// Field numbers for google.protobuf.EnumDescriptorProto.EnumReservedRange. +const ( + EnumDescriptorProto_EnumReservedRange_Start_field_number protoreflect.FieldNumber = 1 + EnumDescriptorProto_EnumReservedRange_End_field_number protoreflect.FieldNumber = 2 +) + +// Names for google.protobuf.EnumValueDescriptorProto. +const ( + EnumValueDescriptorProto_message_name protoreflect.Name = "EnumValueDescriptorProto" + EnumValueDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.EnumValueDescriptorProto" +) + +// Field names for google.protobuf.EnumValueDescriptorProto. +const ( + EnumValueDescriptorProto_Name_field_name protoreflect.Name = "name" + EnumValueDescriptorProto_Number_field_name protoreflect.Name = "number" + EnumValueDescriptorProto_Options_field_name protoreflect.Name = "options" + + EnumValueDescriptorProto_Name_field_fullname protoreflect.FullName = "google.protobuf.EnumValueDescriptorProto.name" + EnumValueDescriptorProto_Number_field_fullname protoreflect.FullName = "google.protobuf.EnumValueDescriptorProto.number" + EnumValueDescriptorProto_Options_field_fullname protoreflect.FullName = "google.protobuf.EnumValueDescriptorProto.options" +) + +// Field numbers for google.protobuf.EnumValueDescriptorProto. +const ( + EnumValueDescriptorProto_Name_field_number protoreflect.FieldNumber = 1 + EnumValueDescriptorProto_Number_field_number protoreflect.FieldNumber = 2 + EnumValueDescriptorProto_Options_field_number protoreflect.FieldNumber = 3 +) + +// Names for google.protobuf.ServiceDescriptorProto. +const ( + ServiceDescriptorProto_message_name protoreflect.Name = "ServiceDescriptorProto" + ServiceDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.ServiceDescriptorProto" +) + +// Field names for google.protobuf.ServiceDescriptorProto. +const ( + ServiceDescriptorProto_Name_field_name protoreflect.Name = "name" + ServiceDescriptorProto_Method_field_name protoreflect.Name = "method" + ServiceDescriptorProto_Options_field_name protoreflect.Name = "options" + + ServiceDescriptorProto_Name_field_fullname protoreflect.FullName = "google.protobuf.ServiceDescriptorProto.name" + ServiceDescriptorProto_Method_field_fullname protoreflect.FullName = "google.protobuf.ServiceDescriptorProto.method" + ServiceDescriptorProto_Options_field_fullname protoreflect.FullName = "google.protobuf.ServiceDescriptorProto.options" +) + +// Field numbers for google.protobuf.ServiceDescriptorProto. +const ( + ServiceDescriptorProto_Name_field_number protoreflect.FieldNumber = 1 + ServiceDescriptorProto_Method_field_number protoreflect.FieldNumber = 2 + ServiceDescriptorProto_Options_field_number protoreflect.FieldNumber = 3 +) + +// Names for google.protobuf.MethodDescriptorProto. +const ( + MethodDescriptorProto_message_name protoreflect.Name = "MethodDescriptorProto" + MethodDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.MethodDescriptorProto" +) + +// Field names for google.protobuf.MethodDescriptorProto. +const ( + MethodDescriptorProto_Name_field_name protoreflect.Name = "name" + MethodDescriptorProto_InputType_field_name protoreflect.Name = "input_type" + MethodDescriptorProto_OutputType_field_name protoreflect.Name = "output_type" + MethodDescriptorProto_Options_field_name protoreflect.Name = "options" + MethodDescriptorProto_ClientStreaming_field_name protoreflect.Name = "client_streaming" + MethodDescriptorProto_ServerStreaming_field_name protoreflect.Name = "server_streaming" + + MethodDescriptorProto_Name_field_fullname protoreflect.FullName = "google.protobuf.MethodDescriptorProto.name" + MethodDescriptorProto_InputType_field_fullname protoreflect.FullName = "google.protobuf.MethodDescriptorProto.input_type" + MethodDescriptorProto_OutputType_field_fullname protoreflect.FullName = "google.protobuf.MethodDescriptorProto.output_type" + MethodDescriptorProto_Options_field_fullname protoreflect.FullName = "google.protobuf.MethodDescriptorProto.options" + MethodDescriptorProto_ClientStreaming_field_fullname protoreflect.FullName = "google.protobuf.MethodDescriptorProto.client_streaming" + MethodDescriptorProto_ServerStreaming_field_fullname protoreflect.FullName = "google.protobuf.MethodDescriptorProto.server_streaming" +) + +// Field numbers for google.protobuf.MethodDescriptorProto. +const ( + MethodDescriptorProto_Name_field_number protoreflect.FieldNumber = 1 + MethodDescriptorProto_InputType_field_number protoreflect.FieldNumber = 2 + MethodDescriptorProto_OutputType_field_number protoreflect.FieldNumber = 3 + MethodDescriptorProto_Options_field_number protoreflect.FieldNumber = 4 + MethodDescriptorProto_ClientStreaming_field_number protoreflect.FieldNumber = 5 + MethodDescriptorProto_ServerStreaming_field_number protoreflect.FieldNumber = 6 +) + +// Names for google.protobuf.FileOptions. +const ( + FileOptions_message_name protoreflect.Name = "FileOptions" + FileOptions_message_fullname protoreflect.FullName = "google.protobuf.FileOptions" +) + +// Field names for google.protobuf.FileOptions. +const ( + FileOptions_JavaPackage_field_name protoreflect.Name = "java_package" + FileOptions_JavaOuterClassname_field_name protoreflect.Name = "java_outer_classname" + FileOptions_JavaMultipleFiles_field_name protoreflect.Name = "java_multiple_files" + FileOptions_JavaGenerateEqualsAndHash_field_name protoreflect.Name = "java_generate_equals_and_hash" + FileOptions_JavaStringCheckUtf8_field_name protoreflect.Name = "java_string_check_utf8" + FileOptions_OptimizeFor_field_name protoreflect.Name = "optimize_for" + FileOptions_GoPackage_field_name protoreflect.Name = "go_package" + FileOptions_CcGenericServices_field_name protoreflect.Name = "cc_generic_services" + FileOptions_JavaGenericServices_field_name protoreflect.Name = "java_generic_services" + FileOptions_PyGenericServices_field_name protoreflect.Name = "py_generic_services" + FileOptions_PhpGenericServices_field_name protoreflect.Name = "php_generic_services" + FileOptions_Deprecated_field_name protoreflect.Name = "deprecated" + FileOptions_CcEnableArenas_field_name protoreflect.Name = "cc_enable_arenas" + FileOptions_ObjcClassPrefix_field_name protoreflect.Name = "objc_class_prefix" + FileOptions_CsharpNamespace_field_name protoreflect.Name = "csharp_namespace" + FileOptions_SwiftPrefix_field_name protoreflect.Name = "swift_prefix" + FileOptions_PhpClassPrefix_field_name protoreflect.Name = "php_class_prefix" + FileOptions_PhpNamespace_field_name protoreflect.Name = "php_namespace" + FileOptions_PhpMetadataNamespace_field_name protoreflect.Name = "php_metadata_namespace" + FileOptions_RubyPackage_field_name protoreflect.Name = "ruby_package" + FileOptions_Features_field_name protoreflect.Name = "features" + FileOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option" + + FileOptions_JavaPackage_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.java_package" + FileOptions_JavaOuterClassname_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.java_outer_classname" + FileOptions_JavaMultipleFiles_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.java_multiple_files" + FileOptions_JavaGenerateEqualsAndHash_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.java_generate_equals_and_hash" + FileOptions_JavaStringCheckUtf8_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.java_string_check_utf8" + FileOptions_OptimizeFor_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.optimize_for" + FileOptions_GoPackage_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.go_package" + FileOptions_CcGenericServices_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.cc_generic_services" + FileOptions_JavaGenericServices_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.java_generic_services" + FileOptions_PyGenericServices_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.py_generic_services" + FileOptions_PhpGenericServices_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.php_generic_services" + FileOptions_Deprecated_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.deprecated" + FileOptions_CcEnableArenas_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.cc_enable_arenas" + FileOptions_ObjcClassPrefix_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.objc_class_prefix" + FileOptions_CsharpNamespace_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.csharp_namespace" + FileOptions_SwiftPrefix_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.swift_prefix" + FileOptions_PhpClassPrefix_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.php_class_prefix" + FileOptions_PhpNamespace_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.php_namespace" + FileOptions_PhpMetadataNamespace_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.php_metadata_namespace" + FileOptions_RubyPackage_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.ruby_package" + FileOptions_Features_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.features" + FileOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.uninterpreted_option" +) + +// Field numbers for google.protobuf.FileOptions. +const ( + FileOptions_JavaPackage_field_number protoreflect.FieldNumber = 1 + FileOptions_JavaOuterClassname_field_number protoreflect.FieldNumber = 8 + FileOptions_JavaMultipleFiles_field_number protoreflect.FieldNumber = 10 + FileOptions_JavaGenerateEqualsAndHash_field_number protoreflect.FieldNumber = 20 + FileOptions_JavaStringCheckUtf8_field_number protoreflect.FieldNumber = 27 + FileOptions_OptimizeFor_field_number protoreflect.FieldNumber = 9 + FileOptions_GoPackage_field_number protoreflect.FieldNumber = 11 + FileOptions_CcGenericServices_field_number protoreflect.FieldNumber = 16 + FileOptions_JavaGenericServices_field_number protoreflect.FieldNumber = 17 + FileOptions_PyGenericServices_field_number protoreflect.FieldNumber = 18 + FileOptions_PhpGenericServices_field_number protoreflect.FieldNumber = 42 + FileOptions_Deprecated_field_number protoreflect.FieldNumber = 23 + FileOptions_CcEnableArenas_field_number protoreflect.FieldNumber = 31 + FileOptions_ObjcClassPrefix_field_number protoreflect.FieldNumber = 36 + FileOptions_CsharpNamespace_field_number protoreflect.FieldNumber = 37 + FileOptions_SwiftPrefix_field_number protoreflect.FieldNumber = 39 + FileOptions_PhpClassPrefix_field_number protoreflect.FieldNumber = 40 + FileOptions_PhpNamespace_field_number protoreflect.FieldNumber = 41 + FileOptions_PhpMetadataNamespace_field_number protoreflect.FieldNumber = 44 + FileOptions_RubyPackage_field_number protoreflect.FieldNumber = 45 + FileOptions_Features_field_number protoreflect.FieldNumber = 50 + FileOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999 +) + +// Full and short names for google.protobuf.FileOptions.OptimizeMode. +const ( + FileOptions_OptimizeMode_enum_fullname = "google.protobuf.FileOptions.OptimizeMode" + FileOptions_OptimizeMode_enum_name = "OptimizeMode" +) + +// Names for google.protobuf.MessageOptions. +const ( + MessageOptions_message_name protoreflect.Name = "MessageOptions" + MessageOptions_message_fullname protoreflect.FullName = "google.protobuf.MessageOptions" +) + +// Field names for google.protobuf.MessageOptions. +const ( + MessageOptions_MessageSetWireFormat_field_name protoreflect.Name = "message_set_wire_format" + MessageOptions_NoStandardDescriptorAccessor_field_name protoreflect.Name = "no_standard_descriptor_accessor" + MessageOptions_Deprecated_field_name protoreflect.Name = "deprecated" + MessageOptions_MapEntry_field_name protoreflect.Name = "map_entry" + MessageOptions_DeprecatedLegacyJsonFieldConflicts_field_name protoreflect.Name = "deprecated_legacy_json_field_conflicts" + MessageOptions_Features_field_name protoreflect.Name = "features" + MessageOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option" + + MessageOptions_MessageSetWireFormat_field_fullname protoreflect.FullName = "google.protobuf.MessageOptions.message_set_wire_format" + MessageOptions_NoStandardDescriptorAccessor_field_fullname protoreflect.FullName = "google.protobuf.MessageOptions.no_standard_descriptor_accessor" + MessageOptions_Deprecated_field_fullname protoreflect.FullName = "google.protobuf.MessageOptions.deprecated" + MessageOptions_MapEntry_field_fullname protoreflect.FullName = "google.protobuf.MessageOptions.map_entry" + MessageOptions_DeprecatedLegacyJsonFieldConflicts_field_fullname protoreflect.FullName = "google.protobuf.MessageOptions.deprecated_legacy_json_field_conflicts" + MessageOptions_Features_field_fullname protoreflect.FullName = "google.protobuf.MessageOptions.features" + MessageOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.MessageOptions.uninterpreted_option" +) + +// Field numbers for google.protobuf.MessageOptions. +const ( + MessageOptions_MessageSetWireFormat_field_number protoreflect.FieldNumber = 1 + MessageOptions_NoStandardDescriptorAccessor_field_number protoreflect.FieldNumber = 2 + MessageOptions_Deprecated_field_number protoreflect.FieldNumber = 3 + MessageOptions_MapEntry_field_number protoreflect.FieldNumber = 7 + MessageOptions_DeprecatedLegacyJsonFieldConflicts_field_number protoreflect.FieldNumber = 11 + MessageOptions_Features_field_number protoreflect.FieldNumber = 12 + MessageOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999 +) + +// Names for google.protobuf.FieldOptions. +const ( + FieldOptions_message_name protoreflect.Name = "FieldOptions" + FieldOptions_message_fullname protoreflect.FullName = "google.protobuf.FieldOptions" +) + +// Field names for google.protobuf.FieldOptions. +const ( + FieldOptions_Ctype_field_name protoreflect.Name = "ctype" + FieldOptions_Packed_field_name protoreflect.Name = "packed" + FieldOptions_Jstype_field_name protoreflect.Name = "jstype" + FieldOptions_Lazy_field_name protoreflect.Name = "lazy" + FieldOptions_UnverifiedLazy_field_name protoreflect.Name = "unverified_lazy" + FieldOptions_Deprecated_field_name protoreflect.Name = "deprecated" + FieldOptions_Weak_field_name protoreflect.Name = "weak" + FieldOptions_DebugRedact_field_name protoreflect.Name = "debug_redact" + FieldOptions_Retention_field_name protoreflect.Name = "retention" + FieldOptions_Targets_field_name protoreflect.Name = "targets" + FieldOptions_EditionDefaults_field_name protoreflect.Name = "edition_defaults" + FieldOptions_Features_field_name protoreflect.Name = "features" + FieldOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option" + + FieldOptions_Ctype_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.ctype" + FieldOptions_Packed_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.packed" + FieldOptions_Jstype_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.jstype" + FieldOptions_Lazy_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.lazy" + FieldOptions_UnverifiedLazy_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.unverified_lazy" + FieldOptions_Deprecated_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.deprecated" + FieldOptions_Weak_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.weak" + FieldOptions_DebugRedact_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.debug_redact" + FieldOptions_Retention_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.retention" + FieldOptions_Targets_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.targets" + FieldOptions_EditionDefaults_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.edition_defaults" + FieldOptions_Features_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.features" + FieldOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.uninterpreted_option" +) + +// Field numbers for google.protobuf.FieldOptions. +const ( + FieldOptions_Ctype_field_number protoreflect.FieldNumber = 1 + FieldOptions_Packed_field_number protoreflect.FieldNumber = 2 + FieldOptions_Jstype_field_number protoreflect.FieldNumber = 6 + FieldOptions_Lazy_field_number protoreflect.FieldNumber = 5 + FieldOptions_UnverifiedLazy_field_number protoreflect.FieldNumber = 15 + FieldOptions_Deprecated_field_number protoreflect.FieldNumber = 3 + FieldOptions_Weak_field_number protoreflect.FieldNumber = 10 + FieldOptions_DebugRedact_field_number protoreflect.FieldNumber = 16 + FieldOptions_Retention_field_number protoreflect.FieldNumber = 17 + FieldOptions_Targets_field_number protoreflect.FieldNumber = 19 + FieldOptions_EditionDefaults_field_number protoreflect.FieldNumber = 20 + FieldOptions_Features_field_number protoreflect.FieldNumber = 21 + FieldOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999 +) + +// Full and short names for google.protobuf.FieldOptions.CType. +const ( + FieldOptions_CType_enum_fullname = "google.protobuf.FieldOptions.CType" + FieldOptions_CType_enum_name = "CType" +) + +// Full and short names for google.protobuf.FieldOptions.JSType. +const ( + FieldOptions_JSType_enum_fullname = "google.protobuf.FieldOptions.JSType" + FieldOptions_JSType_enum_name = "JSType" +) + +// Full and short names for google.protobuf.FieldOptions.OptionRetention. +const ( + FieldOptions_OptionRetention_enum_fullname = "google.protobuf.FieldOptions.OptionRetention" + FieldOptions_OptionRetention_enum_name = "OptionRetention" +) + +// Full and short names for google.protobuf.FieldOptions.OptionTargetType. +const ( + FieldOptions_OptionTargetType_enum_fullname = "google.protobuf.FieldOptions.OptionTargetType" + FieldOptions_OptionTargetType_enum_name = "OptionTargetType" +) + +// Names for google.protobuf.FieldOptions.EditionDefault. +const ( + FieldOptions_EditionDefault_message_name protoreflect.Name = "EditionDefault" + FieldOptions_EditionDefault_message_fullname protoreflect.FullName = "google.protobuf.FieldOptions.EditionDefault" +) + +// Field names for google.protobuf.FieldOptions.EditionDefault. +const ( + FieldOptions_EditionDefault_Edition_field_name protoreflect.Name = "edition" + FieldOptions_EditionDefault_Value_field_name protoreflect.Name = "value" + + FieldOptions_EditionDefault_Edition_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.EditionDefault.edition" + FieldOptions_EditionDefault_Value_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.EditionDefault.value" +) + +// Field numbers for google.protobuf.FieldOptions.EditionDefault. +const ( + FieldOptions_EditionDefault_Edition_field_number protoreflect.FieldNumber = 3 + FieldOptions_EditionDefault_Value_field_number protoreflect.FieldNumber = 2 +) + +// Names for google.protobuf.OneofOptions. +const ( + OneofOptions_message_name protoreflect.Name = "OneofOptions" + OneofOptions_message_fullname protoreflect.FullName = "google.protobuf.OneofOptions" +) + +// Field names for google.protobuf.OneofOptions. +const ( + OneofOptions_Features_field_name protoreflect.Name = "features" + OneofOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option" + + OneofOptions_Features_field_fullname protoreflect.FullName = "google.protobuf.OneofOptions.features" + OneofOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.OneofOptions.uninterpreted_option" +) + +// Field numbers for google.protobuf.OneofOptions. +const ( + OneofOptions_Features_field_number protoreflect.FieldNumber = 1 + OneofOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999 +) + +// Names for google.protobuf.EnumOptions. +const ( + EnumOptions_message_name protoreflect.Name = "EnumOptions" + EnumOptions_message_fullname protoreflect.FullName = "google.protobuf.EnumOptions" +) + +// Field names for google.protobuf.EnumOptions. +const ( + EnumOptions_AllowAlias_field_name protoreflect.Name = "allow_alias" + EnumOptions_Deprecated_field_name protoreflect.Name = "deprecated" + EnumOptions_DeprecatedLegacyJsonFieldConflicts_field_name protoreflect.Name = "deprecated_legacy_json_field_conflicts" + EnumOptions_Features_field_name protoreflect.Name = "features" + EnumOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option" + + EnumOptions_AllowAlias_field_fullname protoreflect.FullName = "google.protobuf.EnumOptions.allow_alias" + EnumOptions_Deprecated_field_fullname protoreflect.FullName = "google.protobuf.EnumOptions.deprecated" + EnumOptions_DeprecatedLegacyJsonFieldConflicts_field_fullname protoreflect.FullName = "google.protobuf.EnumOptions.deprecated_legacy_json_field_conflicts" + EnumOptions_Features_field_fullname protoreflect.FullName = "google.protobuf.EnumOptions.features" + EnumOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.EnumOptions.uninterpreted_option" +) + +// Field numbers for google.protobuf.EnumOptions. +const ( + EnumOptions_AllowAlias_field_number protoreflect.FieldNumber = 2 + EnumOptions_Deprecated_field_number protoreflect.FieldNumber = 3 + EnumOptions_DeprecatedLegacyJsonFieldConflicts_field_number protoreflect.FieldNumber = 6 + EnumOptions_Features_field_number protoreflect.FieldNumber = 7 + EnumOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999 +) + +// Names for google.protobuf.EnumValueOptions. +const ( + EnumValueOptions_message_name protoreflect.Name = "EnumValueOptions" + EnumValueOptions_message_fullname protoreflect.FullName = "google.protobuf.EnumValueOptions" +) + +// Field names for google.protobuf.EnumValueOptions. +const ( + EnumValueOptions_Deprecated_field_name protoreflect.Name = "deprecated" + EnumValueOptions_Features_field_name protoreflect.Name = "features" + EnumValueOptions_DebugRedact_field_name protoreflect.Name = "debug_redact" + EnumValueOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option" + + EnumValueOptions_Deprecated_field_fullname protoreflect.FullName = "google.protobuf.EnumValueOptions.deprecated" + EnumValueOptions_Features_field_fullname protoreflect.FullName = "google.protobuf.EnumValueOptions.features" + EnumValueOptions_DebugRedact_field_fullname protoreflect.FullName = "google.protobuf.EnumValueOptions.debug_redact" + EnumValueOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.EnumValueOptions.uninterpreted_option" +) + +// Field numbers for google.protobuf.EnumValueOptions. +const ( + EnumValueOptions_Deprecated_field_number protoreflect.FieldNumber = 1 + EnumValueOptions_Features_field_number protoreflect.FieldNumber = 2 + EnumValueOptions_DebugRedact_field_number protoreflect.FieldNumber = 3 + EnumValueOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999 +) + +// Names for google.protobuf.ServiceOptions. +const ( + ServiceOptions_message_name protoreflect.Name = "ServiceOptions" + ServiceOptions_message_fullname protoreflect.FullName = "google.protobuf.ServiceOptions" +) + +// Field names for google.protobuf.ServiceOptions. +const ( + ServiceOptions_Features_field_name protoreflect.Name = "features" + ServiceOptions_Deprecated_field_name protoreflect.Name = "deprecated" + ServiceOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option" + + ServiceOptions_Features_field_fullname protoreflect.FullName = "google.protobuf.ServiceOptions.features" + ServiceOptions_Deprecated_field_fullname protoreflect.FullName = "google.protobuf.ServiceOptions.deprecated" + ServiceOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.ServiceOptions.uninterpreted_option" +) + +// Field numbers for google.protobuf.ServiceOptions. +const ( + ServiceOptions_Features_field_number protoreflect.FieldNumber = 34 + ServiceOptions_Deprecated_field_number protoreflect.FieldNumber = 33 + ServiceOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999 +) + +// Names for google.protobuf.MethodOptions. +const ( + MethodOptions_message_name protoreflect.Name = "MethodOptions" + MethodOptions_message_fullname protoreflect.FullName = "google.protobuf.MethodOptions" +) + +// Field names for google.protobuf.MethodOptions. +const ( + MethodOptions_Deprecated_field_name protoreflect.Name = "deprecated" + MethodOptions_IdempotencyLevel_field_name protoreflect.Name = "idempotency_level" + MethodOptions_Features_field_name protoreflect.Name = "features" + MethodOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option" + + MethodOptions_Deprecated_field_fullname protoreflect.FullName = "google.protobuf.MethodOptions.deprecated" + MethodOptions_IdempotencyLevel_field_fullname protoreflect.FullName = "google.protobuf.MethodOptions.idempotency_level" + MethodOptions_Features_field_fullname protoreflect.FullName = "google.protobuf.MethodOptions.features" + MethodOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.MethodOptions.uninterpreted_option" +) + +// Field numbers for google.protobuf.MethodOptions. +const ( + MethodOptions_Deprecated_field_number protoreflect.FieldNumber = 33 + MethodOptions_IdempotencyLevel_field_number protoreflect.FieldNumber = 34 + MethodOptions_Features_field_number protoreflect.FieldNumber = 35 + MethodOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999 +) + +// Full and short names for google.protobuf.MethodOptions.IdempotencyLevel. +const ( + MethodOptions_IdempotencyLevel_enum_fullname = "google.protobuf.MethodOptions.IdempotencyLevel" + MethodOptions_IdempotencyLevel_enum_name = "IdempotencyLevel" +) + +// Names for google.protobuf.UninterpretedOption. +const ( + UninterpretedOption_message_name protoreflect.Name = "UninterpretedOption" + UninterpretedOption_message_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption" +) + +// Field names for google.protobuf.UninterpretedOption. +const ( + UninterpretedOption_Name_field_name protoreflect.Name = "name" + UninterpretedOption_IdentifierValue_field_name protoreflect.Name = "identifier_value" + UninterpretedOption_PositiveIntValue_field_name protoreflect.Name = "positive_int_value" + UninterpretedOption_NegativeIntValue_field_name protoreflect.Name = "negative_int_value" + UninterpretedOption_DoubleValue_field_name protoreflect.Name = "double_value" + UninterpretedOption_StringValue_field_name protoreflect.Name = "string_value" + UninterpretedOption_AggregateValue_field_name protoreflect.Name = "aggregate_value" + + UninterpretedOption_Name_field_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.name" + UninterpretedOption_IdentifierValue_field_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.identifier_value" + UninterpretedOption_PositiveIntValue_field_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.positive_int_value" + UninterpretedOption_NegativeIntValue_field_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.negative_int_value" + UninterpretedOption_DoubleValue_field_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.double_value" + UninterpretedOption_StringValue_field_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.string_value" + UninterpretedOption_AggregateValue_field_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.aggregate_value" +) + +// Field numbers for google.protobuf.UninterpretedOption. +const ( + UninterpretedOption_Name_field_number protoreflect.FieldNumber = 2 + UninterpretedOption_IdentifierValue_field_number protoreflect.FieldNumber = 3 + UninterpretedOption_PositiveIntValue_field_number protoreflect.FieldNumber = 4 + UninterpretedOption_NegativeIntValue_field_number protoreflect.FieldNumber = 5 + UninterpretedOption_DoubleValue_field_number protoreflect.FieldNumber = 6 + UninterpretedOption_StringValue_field_number protoreflect.FieldNumber = 7 + UninterpretedOption_AggregateValue_field_number protoreflect.FieldNumber = 8 +) + +// Names for google.protobuf.UninterpretedOption.NamePart. +const ( + UninterpretedOption_NamePart_message_name protoreflect.Name = "NamePart" + UninterpretedOption_NamePart_message_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.NamePart" +) + +// Field names for google.protobuf.UninterpretedOption.NamePart. +const ( + UninterpretedOption_NamePart_NamePart_field_name protoreflect.Name = "name_part" + UninterpretedOption_NamePart_IsExtension_field_name protoreflect.Name = "is_extension" + + UninterpretedOption_NamePart_NamePart_field_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.NamePart.name_part" + UninterpretedOption_NamePart_IsExtension_field_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.NamePart.is_extension" +) + +// Field numbers for google.protobuf.UninterpretedOption.NamePart. +const ( + UninterpretedOption_NamePart_NamePart_field_number protoreflect.FieldNumber = 1 + UninterpretedOption_NamePart_IsExtension_field_number protoreflect.FieldNumber = 2 +) + +// Names for google.protobuf.FeatureSet. +const ( + FeatureSet_message_name protoreflect.Name = "FeatureSet" + FeatureSet_message_fullname protoreflect.FullName = "google.protobuf.FeatureSet" +) + +// Field names for google.protobuf.FeatureSet. +const ( + FeatureSet_FieldPresence_field_name protoreflect.Name = "field_presence" + FeatureSet_EnumType_field_name protoreflect.Name = "enum_type" + FeatureSet_RepeatedFieldEncoding_field_name protoreflect.Name = "repeated_field_encoding" + FeatureSet_Utf8Validation_field_name protoreflect.Name = "utf8_validation" + FeatureSet_MessageEncoding_field_name protoreflect.Name = "message_encoding" + FeatureSet_JsonFormat_field_name protoreflect.Name = "json_format" + + FeatureSet_FieldPresence_field_fullname protoreflect.FullName = "google.protobuf.FeatureSet.field_presence" + FeatureSet_EnumType_field_fullname protoreflect.FullName = "google.protobuf.FeatureSet.enum_type" + FeatureSet_RepeatedFieldEncoding_field_fullname protoreflect.FullName = "google.protobuf.FeatureSet.repeated_field_encoding" + FeatureSet_Utf8Validation_field_fullname protoreflect.FullName = "google.protobuf.FeatureSet.utf8_validation" + FeatureSet_MessageEncoding_field_fullname protoreflect.FullName = "google.protobuf.FeatureSet.message_encoding" + FeatureSet_JsonFormat_field_fullname protoreflect.FullName = "google.protobuf.FeatureSet.json_format" +) + +// Field numbers for google.protobuf.FeatureSet. +const ( + FeatureSet_FieldPresence_field_number protoreflect.FieldNumber = 1 + FeatureSet_EnumType_field_number protoreflect.FieldNumber = 2 + FeatureSet_RepeatedFieldEncoding_field_number protoreflect.FieldNumber = 3 + FeatureSet_Utf8Validation_field_number protoreflect.FieldNumber = 4 + FeatureSet_MessageEncoding_field_number protoreflect.FieldNumber = 5 + FeatureSet_JsonFormat_field_number protoreflect.FieldNumber = 6 +) + +// Full and short names for google.protobuf.FeatureSet.FieldPresence. +const ( + FeatureSet_FieldPresence_enum_fullname = "google.protobuf.FeatureSet.FieldPresence" + FeatureSet_FieldPresence_enum_name = "FieldPresence" +) + +// Full and short names for google.protobuf.FeatureSet.EnumType. +const ( + FeatureSet_EnumType_enum_fullname = "google.protobuf.FeatureSet.EnumType" + FeatureSet_EnumType_enum_name = "EnumType" +) + +// Full and short names for google.protobuf.FeatureSet.RepeatedFieldEncoding. +const ( + FeatureSet_RepeatedFieldEncoding_enum_fullname = "google.protobuf.FeatureSet.RepeatedFieldEncoding" + FeatureSet_RepeatedFieldEncoding_enum_name = "RepeatedFieldEncoding" +) + +// Full and short names for google.protobuf.FeatureSet.Utf8Validation. +const ( + FeatureSet_Utf8Validation_enum_fullname = "google.protobuf.FeatureSet.Utf8Validation" + FeatureSet_Utf8Validation_enum_name = "Utf8Validation" +) + +// Full and short names for google.protobuf.FeatureSet.MessageEncoding. +const ( + FeatureSet_MessageEncoding_enum_fullname = "google.protobuf.FeatureSet.MessageEncoding" + FeatureSet_MessageEncoding_enum_name = "MessageEncoding" +) + +// Full and short names for google.protobuf.FeatureSet.JsonFormat. +const ( + FeatureSet_JsonFormat_enum_fullname = "google.protobuf.FeatureSet.JsonFormat" + FeatureSet_JsonFormat_enum_name = "JsonFormat" +) + +// Names for google.protobuf.FeatureSetDefaults. +const ( + FeatureSetDefaults_message_name protoreflect.Name = "FeatureSetDefaults" + FeatureSetDefaults_message_fullname protoreflect.FullName = "google.protobuf.FeatureSetDefaults" +) + +// Field names for google.protobuf.FeatureSetDefaults. +const ( + FeatureSetDefaults_Defaults_field_name protoreflect.Name = "defaults" + FeatureSetDefaults_MinimumEdition_field_name protoreflect.Name = "minimum_edition" + FeatureSetDefaults_MaximumEdition_field_name protoreflect.Name = "maximum_edition" + + FeatureSetDefaults_Defaults_field_fullname protoreflect.FullName = "google.protobuf.FeatureSetDefaults.defaults" + FeatureSetDefaults_MinimumEdition_field_fullname protoreflect.FullName = "google.protobuf.FeatureSetDefaults.minimum_edition" + FeatureSetDefaults_MaximumEdition_field_fullname protoreflect.FullName = "google.protobuf.FeatureSetDefaults.maximum_edition" +) + +// Field numbers for google.protobuf.FeatureSetDefaults. +const ( + FeatureSetDefaults_Defaults_field_number protoreflect.FieldNumber = 1 + FeatureSetDefaults_MinimumEdition_field_number protoreflect.FieldNumber = 4 + FeatureSetDefaults_MaximumEdition_field_number protoreflect.FieldNumber = 5 +) + +// Names for google.protobuf.FeatureSetDefaults.FeatureSetEditionDefault. +const ( + FeatureSetDefaults_FeatureSetEditionDefault_message_name protoreflect.Name = "FeatureSetEditionDefault" + FeatureSetDefaults_FeatureSetEditionDefault_message_fullname protoreflect.FullName = "google.protobuf.FeatureSetDefaults.FeatureSetEditionDefault" +) + +// Field names for google.protobuf.FeatureSetDefaults.FeatureSetEditionDefault. +const ( + FeatureSetDefaults_FeatureSetEditionDefault_Edition_field_name protoreflect.Name = "edition" + FeatureSetDefaults_FeatureSetEditionDefault_Features_field_name protoreflect.Name = "features" + + FeatureSetDefaults_FeatureSetEditionDefault_Edition_field_fullname protoreflect.FullName = "google.protobuf.FeatureSetDefaults.FeatureSetEditionDefault.edition" + FeatureSetDefaults_FeatureSetEditionDefault_Features_field_fullname protoreflect.FullName = "google.protobuf.FeatureSetDefaults.FeatureSetEditionDefault.features" +) + +// Field numbers for google.protobuf.FeatureSetDefaults.FeatureSetEditionDefault. +const ( + FeatureSetDefaults_FeatureSetEditionDefault_Edition_field_number protoreflect.FieldNumber = 3 + FeatureSetDefaults_FeatureSetEditionDefault_Features_field_number protoreflect.FieldNumber = 2 +) + +// Names for google.protobuf.SourceCodeInfo. +const ( + SourceCodeInfo_message_name protoreflect.Name = "SourceCodeInfo" + SourceCodeInfo_message_fullname protoreflect.FullName = "google.protobuf.SourceCodeInfo" +) + +// Field names for google.protobuf.SourceCodeInfo. +const ( + SourceCodeInfo_Location_field_name protoreflect.Name = "location" + + SourceCodeInfo_Location_field_fullname protoreflect.FullName = "google.protobuf.SourceCodeInfo.location" +) + +// Field numbers for google.protobuf.SourceCodeInfo. +const ( + SourceCodeInfo_Location_field_number protoreflect.FieldNumber = 1 +) + +// Names for google.protobuf.SourceCodeInfo.Location. +const ( + SourceCodeInfo_Location_message_name protoreflect.Name = "Location" + SourceCodeInfo_Location_message_fullname protoreflect.FullName = "google.protobuf.SourceCodeInfo.Location" +) + +// Field names for google.protobuf.SourceCodeInfo.Location. +const ( + SourceCodeInfo_Location_Path_field_name protoreflect.Name = "path" + SourceCodeInfo_Location_Span_field_name protoreflect.Name = "span" + SourceCodeInfo_Location_LeadingComments_field_name protoreflect.Name = "leading_comments" + SourceCodeInfo_Location_TrailingComments_field_name protoreflect.Name = "trailing_comments" + SourceCodeInfo_Location_LeadingDetachedComments_field_name protoreflect.Name = "leading_detached_comments" + + SourceCodeInfo_Location_Path_field_fullname protoreflect.FullName = "google.protobuf.SourceCodeInfo.Location.path" + SourceCodeInfo_Location_Span_field_fullname protoreflect.FullName = "google.protobuf.SourceCodeInfo.Location.span" + SourceCodeInfo_Location_LeadingComments_field_fullname protoreflect.FullName = "google.protobuf.SourceCodeInfo.Location.leading_comments" + SourceCodeInfo_Location_TrailingComments_field_fullname protoreflect.FullName = "google.protobuf.SourceCodeInfo.Location.trailing_comments" + SourceCodeInfo_Location_LeadingDetachedComments_field_fullname protoreflect.FullName = "google.protobuf.SourceCodeInfo.Location.leading_detached_comments" +) + +// Field numbers for google.protobuf.SourceCodeInfo.Location. +const ( + SourceCodeInfo_Location_Path_field_number protoreflect.FieldNumber = 1 + SourceCodeInfo_Location_Span_field_number protoreflect.FieldNumber = 2 + SourceCodeInfo_Location_LeadingComments_field_number protoreflect.FieldNumber = 3 + SourceCodeInfo_Location_TrailingComments_field_number protoreflect.FieldNumber = 4 + SourceCodeInfo_Location_LeadingDetachedComments_field_number protoreflect.FieldNumber = 6 +) + +// Names for google.protobuf.GeneratedCodeInfo. +const ( + GeneratedCodeInfo_message_name protoreflect.Name = "GeneratedCodeInfo" + GeneratedCodeInfo_message_fullname protoreflect.FullName = "google.protobuf.GeneratedCodeInfo" +) + +// Field names for google.protobuf.GeneratedCodeInfo. +const ( + GeneratedCodeInfo_Annotation_field_name protoreflect.Name = "annotation" + + GeneratedCodeInfo_Annotation_field_fullname protoreflect.FullName = "google.protobuf.GeneratedCodeInfo.annotation" +) + +// Field numbers for google.protobuf.GeneratedCodeInfo. +const ( + GeneratedCodeInfo_Annotation_field_number protoreflect.FieldNumber = 1 +) + +// Names for google.protobuf.GeneratedCodeInfo.Annotation. +const ( + GeneratedCodeInfo_Annotation_message_name protoreflect.Name = "Annotation" + GeneratedCodeInfo_Annotation_message_fullname protoreflect.FullName = "google.protobuf.GeneratedCodeInfo.Annotation" +) + +// Field names for google.protobuf.GeneratedCodeInfo.Annotation. +const ( + GeneratedCodeInfo_Annotation_Path_field_name protoreflect.Name = "path" + GeneratedCodeInfo_Annotation_SourceFile_field_name protoreflect.Name = "source_file" + GeneratedCodeInfo_Annotation_Begin_field_name protoreflect.Name = "begin" + GeneratedCodeInfo_Annotation_End_field_name protoreflect.Name = "end" + GeneratedCodeInfo_Annotation_Semantic_field_name protoreflect.Name = "semantic" + + GeneratedCodeInfo_Annotation_Path_field_fullname protoreflect.FullName = "google.protobuf.GeneratedCodeInfo.Annotation.path" + GeneratedCodeInfo_Annotation_SourceFile_field_fullname protoreflect.FullName = "google.protobuf.GeneratedCodeInfo.Annotation.source_file" + GeneratedCodeInfo_Annotation_Begin_field_fullname protoreflect.FullName = "google.protobuf.GeneratedCodeInfo.Annotation.begin" + GeneratedCodeInfo_Annotation_End_field_fullname protoreflect.FullName = "google.protobuf.GeneratedCodeInfo.Annotation.end" + GeneratedCodeInfo_Annotation_Semantic_field_fullname protoreflect.FullName = "google.protobuf.GeneratedCodeInfo.Annotation.semantic" +) + +// Field numbers for google.protobuf.GeneratedCodeInfo.Annotation. +const ( + GeneratedCodeInfo_Annotation_Path_field_number protoreflect.FieldNumber = 1 + GeneratedCodeInfo_Annotation_SourceFile_field_number protoreflect.FieldNumber = 2 + GeneratedCodeInfo_Annotation_Begin_field_number protoreflect.FieldNumber = 3 + GeneratedCodeInfo_Annotation_End_field_number protoreflect.FieldNumber = 4 + GeneratedCodeInfo_Annotation_Semantic_field_number protoreflect.FieldNumber = 5 +) + +// Full and short names for google.protobuf.GeneratedCodeInfo.Annotation.Semantic. +const ( + GeneratedCodeInfo_Annotation_Semantic_enum_fullname = "google.protobuf.GeneratedCodeInfo.Annotation.Semantic" + GeneratedCodeInfo_Annotation_Semantic_enum_name = "Semantic" +) diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/doc.go b/vendor/go.temporal.io/api/internal/protojson/genid/doc.go new file mode 100644 index 00000000000..45ccd01211c --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/doc.go @@ -0,0 +1,11 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package genid contains constants for declarations in descriptor.proto +// and the well-known types. +package genid + +import protoreflect "google.golang.org/protobuf/reflect/protoreflect" + +const GoogleProtobuf_package protoreflect.FullName = "google.protobuf" diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/duration_gen.go b/vendor/go.temporal.io/api/internal/protojson/genid/duration_gen.go new file mode 100644 index 00000000000..b070ef4fd69 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/duration_gen.go @@ -0,0 +1,34 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Code generated by generate-protos. DO NOT EDIT. + +package genid + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" +) + +const File_google_protobuf_duration_proto = "google/protobuf/duration.proto" + +// Names for google.protobuf.Duration. +const ( + Duration_message_name protoreflect.Name = "Duration" + Duration_message_fullname protoreflect.FullName = "google.protobuf.Duration" +) + +// Field names for google.protobuf.Duration. +const ( + Duration_Seconds_field_name protoreflect.Name = "seconds" + Duration_Nanos_field_name protoreflect.Name = "nanos" + + Duration_Seconds_field_fullname protoreflect.FullName = "google.protobuf.Duration.seconds" + Duration_Nanos_field_fullname protoreflect.FullName = "google.protobuf.Duration.nanos" +) + +// Field numbers for google.protobuf.Duration. +const ( + Duration_Seconds_field_number protoreflect.FieldNumber = 1 + Duration_Nanos_field_number protoreflect.FieldNumber = 2 +) diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/empty_gen.go b/vendor/go.temporal.io/api/internal/protojson/genid/empty_gen.go new file mode 100644 index 00000000000..762abb34a4e --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/empty_gen.go @@ -0,0 +1,19 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Code generated by generate-protos. DO NOT EDIT. + +package genid + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" +) + +const File_google_protobuf_empty_proto = "google/protobuf/empty.proto" + +// Names for google.protobuf.Empty. +const ( + Empty_message_name protoreflect.Name = "Empty" + Empty_message_fullname protoreflect.FullName = "google.protobuf.Empty" +) diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/field_mask_gen.go b/vendor/go.temporal.io/api/internal/protojson/genid/field_mask_gen.go new file mode 100644 index 00000000000..70bed453fee --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/field_mask_gen.go @@ -0,0 +1,31 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Code generated by generate-protos. DO NOT EDIT. + +package genid + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" +) + +const File_google_protobuf_field_mask_proto = "google/protobuf/field_mask.proto" + +// Names for google.protobuf.FieldMask. +const ( + FieldMask_message_name protoreflect.Name = "FieldMask" + FieldMask_message_fullname protoreflect.FullName = "google.protobuf.FieldMask" +) + +// Field names for google.protobuf.FieldMask. +const ( + FieldMask_Paths_field_name protoreflect.Name = "paths" + + FieldMask_Paths_field_fullname protoreflect.FullName = "google.protobuf.FieldMask.paths" +) + +// Field numbers for google.protobuf.FieldMask. +const ( + FieldMask_Paths_field_number protoreflect.FieldNumber = 1 +) diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/goname.go b/vendor/go.temporal.io/api/internal/protojson/genid/goname.go new file mode 100644 index 00000000000..693d2e9e1fe --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/goname.go @@ -0,0 +1,25 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package genid + +// Go names of implementation-specific struct fields in generated messages. +const ( + State_goname = "state" + + SizeCache_goname = "sizeCache" + SizeCacheA_goname = "XXX_sizecache" + + WeakFields_goname = "weakFields" + WeakFieldsA_goname = "XXX_weak" + + UnknownFields_goname = "unknownFields" + UnknownFieldsA_goname = "XXX_unrecognized" + + ExtensionFields_goname = "extensionFields" + ExtensionFieldsA_goname = "XXX_InternalExtensions" + ExtensionFieldsB_goname = "XXX_extensions" + + WeakFieldPrefix_goname = "XXX_weak_" +) diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/map_entry.go b/vendor/go.temporal.io/api/internal/protojson/genid/map_entry.go new file mode 100644 index 00000000000..8f9ea02ff2a --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/map_entry.go @@ -0,0 +1,16 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package genid + +import protoreflect "google.golang.org/protobuf/reflect/protoreflect" + +// Generic field names and numbers for synthetic map entry messages. +const ( + MapEntry_Key_field_name protoreflect.Name = "key" + MapEntry_Value_field_name protoreflect.Name = "value" + + MapEntry_Key_field_number protoreflect.FieldNumber = 1 + MapEntry_Value_field_number protoreflect.FieldNumber = 2 +) diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/source_context_gen.go b/vendor/go.temporal.io/api/internal/protojson/genid/source_context_gen.go new file mode 100644 index 00000000000..3e99ae16c84 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/source_context_gen.go @@ -0,0 +1,31 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Code generated by generate-protos. DO NOT EDIT. + +package genid + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" +) + +const File_google_protobuf_source_context_proto = "google/protobuf/source_context.proto" + +// Names for google.protobuf.SourceContext. +const ( + SourceContext_message_name protoreflect.Name = "SourceContext" + SourceContext_message_fullname protoreflect.FullName = "google.protobuf.SourceContext" +) + +// Field names for google.protobuf.SourceContext. +const ( + SourceContext_FileName_field_name protoreflect.Name = "file_name" + + SourceContext_FileName_field_fullname protoreflect.FullName = "google.protobuf.SourceContext.file_name" +) + +// Field numbers for google.protobuf.SourceContext. +const ( + SourceContext_FileName_field_number protoreflect.FieldNumber = 1 +) diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/struct_gen.go b/vendor/go.temporal.io/api/internal/protojson/genid/struct_gen.go new file mode 100644 index 00000000000..1a38944b26e --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/struct_gen.go @@ -0,0 +1,116 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Code generated by generate-protos. DO NOT EDIT. + +package genid + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" +) + +const File_google_protobuf_struct_proto = "google/protobuf/struct.proto" + +// Full and short names for google.protobuf.NullValue. +const ( + NullValue_enum_fullname = "google.protobuf.NullValue" + NullValue_enum_name = "NullValue" +) + +// Names for google.protobuf.Struct. +const ( + Struct_message_name protoreflect.Name = "Struct" + Struct_message_fullname protoreflect.FullName = "google.protobuf.Struct" +) + +// Field names for google.protobuf.Struct. +const ( + Struct_Fields_field_name protoreflect.Name = "fields" + + Struct_Fields_field_fullname protoreflect.FullName = "google.protobuf.Struct.fields" +) + +// Field numbers for google.protobuf.Struct. +const ( + Struct_Fields_field_number protoreflect.FieldNumber = 1 +) + +// Names for google.protobuf.Struct.FieldsEntry. +const ( + Struct_FieldsEntry_message_name protoreflect.Name = "FieldsEntry" + Struct_FieldsEntry_message_fullname protoreflect.FullName = "google.protobuf.Struct.FieldsEntry" +) + +// Field names for google.protobuf.Struct.FieldsEntry. +const ( + Struct_FieldsEntry_Key_field_name protoreflect.Name = "key" + Struct_FieldsEntry_Value_field_name protoreflect.Name = "value" + + Struct_FieldsEntry_Key_field_fullname protoreflect.FullName = "google.protobuf.Struct.FieldsEntry.key" + Struct_FieldsEntry_Value_field_fullname protoreflect.FullName = "google.protobuf.Struct.FieldsEntry.value" +) + +// Field numbers for google.protobuf.Struct.FieldsEntry. +const ( + Struct_FieldsEntry_Key_field_number protoreflect.FieldNumber = 1 + Struct_FieldsEntry_Value_field_number protoreflect.FieldNumber = 2 +) + +// Names for google.protobuf.Value. +const ( + Value_message_name protoreflect.Name = "Value" + Value_message_fullname protoreflect.FullName = "google.protobuf.Value" +) + +// Field names for google.protobuf.Value. +const ( + Value_NullValue_field_name protoreflect.Name = "null_value" + Value_NumberValue_field_name protoreflect.Name = "number_value" + Value_StringValue_field_name protoreflect.Name = "string_value" + Value_BoolValue_field_name protoreflect.Name = "bool_value" + Value_StructValue_field_name protoreflect.Name = "struct_value" + Value_ListValue_field_name protoreflect.Name = "list_value" + + Value_NullValue_field_fullname protoreflect.FullName = "google.protobuf.Value.null_value" + Value_NumberValue_field_fullname protoreflect.FullName = "google.protobuf.Value.number_value" + Value_StringValue_field_fullname protoreflect.FullName = "google.protobuf.Value.string_value" + Value_BoolValue_field_fullname protoreflect.FullName = "google.protobuf.Value.bool_value" + Value_StructValue_field_fullname protoreflect.FullName = "google.protobuf.Value.struct_value" + Value_ListValue_field_fullname protoreflect.FullName = "google.protobuf.Value.list_value" +) + +// Field numbers for google.protobuf.Value. +const ( + Value_NullValue_field_number protoreflect.FieldNumber = 1 + Value_NumberValue_field_number protoreflect.FieldNumber = 2 + Value_StringValue_field_number protoreflect.FieldNumber = 3 + Value_BoolValue_field_number protoreflect.FieldNumber = 4 + Value_StructValue_field_number protoreflect.FieldNumber = 5 + Value_ListValue_field_number protoreflect.FieldNumber = 6 +) + +// Oneof names for google.protobuf.Value. +const ( + Value_Kind_oneof_name protoreflect.Name = "kind" + + Value_Kind_oneof_fullname protoreflect.FullName = "google.protobuf.Value.kind" +) + +// Names for google.protobuf.ListValue. +const ( + ListValue_message_name protoreflect.Name = "ListValue" + ListValue_message_fullname protoreflect.FullName = "google.protobuf.ListValue" +) + +// Field names for google.protobuf.ListValue. +const ( + ListValue_Values_field_name protoreflect.Name = "values" + + ListValue_Values_field_fullname protoreflect.FullName = "google.protobuf.ListValue.values" +) + +// Field numbers for google.protobuf.ListValue. +const ( + ListValue_Values_field_number protoreflect.FieldNumber = 1 +) diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/timestamp_gen.go b/vendor/go.temporal.io/api/internal/protojson/genid/timestamp_gen.go new file mode 100644 index 00000000000..f5cd5634c2e --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/timestamp_gen.go @@ -0,0 +1,34 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Code generated by generate-protos. DO NOT EDIT. + +package genid + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" +) + +const File_google_protobuf_timestamp_proto = "google/protobuf/timestamp.proto" + +// Names for google.protobuf.Timestamp. +const ( + Timestamp_message_name protoreflect.Name = "Timestamp" + Timestamp_message_fullname protoreflect.FullName = "google.protobuf.Timestamp" +) + +// Field names for google.protobuf.Timestamp. +const ( + Timestamp_Seconds_field_name protoreflect.Name = "seconds" + Timestamp_Nanos_field_name protoreflect.Name = "nanos" + + Timestamp_Seconds_field_fullname protoreflect.FullName = "google.protobuf.Timestamp.seconds" + Timestamp_Nanos_field_fullname protoreflect.FullName = "google.protobuf.Timestamp.nanos" +) + +// Field numbers for google.protobuf.Timestamp. +const ( + Timestamp_Seconds_field_number protoreflect.FieldNumber = 1 + Timestamp_Nanos_field_number protoreflect.FieldNumber = 2 +) diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/type_gen.go b/vendor/go.temporal.io/api/internal/protojson/genid/type_gen.go new file mode 100644 index 00000000000..e0f75fea0a1 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/type_gen.go @@ -0,0 +1,190 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Code generated by generate-protos. DO NOT EDIT. + +package genid + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" +) + +const File_google_protobuf_type_proto = "google/protobuf/type.proto" + +// Full and short names for google.protobuf.Syntax. +const ( + Syntax_enum_fullname = "google.protobuf.Syntax" + Syntax_enum_name = "Syntax" +) + +// Names for google.protobuf.Type. +const ( + Type_message_name protoreflect.Name = "Type" + Type_message_fullname protoreflect.FullName = "google.protobuf.Type" +) + +// Field names for google.protobuf.Type. +const ( + Type_Name_field_name protoreflect.Name = "name" + Type_Fields_field_name protoreflect.Name = "fields" + Type_Oneofs_field_name protoreflect.Name = "oneofs" + Type_Options_field_name protoreflect.Name = "options" + Type_SourceContext_field_name protoreflect.Name = "source_context" + Type_Syntax_field_name protoreflect.Name = "syntax" + Type_Edition_field_name protoreflect.Name = "edition" + + Type_Name_field_fullname protoreflect.FullName = "google.protobuf.Type.name" + Type_Fields_field_fullname protoreflect.FullName = "google.protobuf.Type.fields" + Type_Oneofs_field_fullname protoreflect.FullName = "google.protobuf.Type.oneofs" + Type_Options_field_fullname protoreflect.FullName = "google.protobuf.Type.options" + Type_SourceContext_field_fullname protoreflect.FullName = "google.protobuf.Type.source_context" + Type_Syntax_field_fullname protoreflect.FullName = "google.protobuf.Type.syntax" + Type_Edition_field_fullname protoreflect.FullName = "google.protobuf.Type.edition" +) + +// Field numbers for google.protobuf.Type. +const ( + Type_Name_field_number protoreflect.FieldNumber = 1 + Type_Fields_field_number protoreflect.FieldNumber = 2 + Type_Oneofs_field_number protoreflect.FieldNumber = 3 + Type_Options_field_number protoreflect.FieldNumber = 4 + Type_SourceContext_field_number protoreflect.FieldNumber = 5 + Type_Syntax_field_number protoreflect.FieldNumber = 6 + Type_Edition_field_number protoreflect.FieldNumber = 7 +) + +// Names for google.protobuf.Field. +const ( + Field_message_name protoreflect.Name = "Field" + Field_message_fullname protoreflect.FullName = "google.protobuf.Field" +) + +// Field names for google.protobuf.Field. +const ( + Field_Kind_field_name protoreflect.Name = "kind" + Field_Cardinality_field_name protoreflect.Name = "cardinality" + Field_Number_field_name protoreflect.Name = "number" + Field_Name_field_name protoreflect.Name = "name" + Field_TypeUrl_field_name protoreflect.Name = "type_url" + Field_OneofIndex_field_name protoreflect.Name = "oneof_index" + Field_Packed_field_name protoreflect.Name = "packed" + Field_Options_field_name protoreflect.Name = "options" + Field_JsonName_field_name protoreflect.Name = "json_name" + Field_DefaultValue_field_name protoreflect.Name = "default_value" + + Field_Kind_field_fullname protoreflect.FullName = "google.protobuf.Field.kind" + Field_Cardinality_field_fullname protoreflect.FullName = "google.protobuf.Field.cardinality" + Field_Number_field_fullname protoreflect.FullName = "google.protobuf.Field.number" + Field_Name_field_fullname protoreflect.FullName = "google.protobuf.Field.name" + Field_TypeUrl_field_fullname protoreflect.FullName = "google.protobuf.Field.type_url" + Field_OneofIndex_field_fullname protoreflect.FullName = "google.protobuf.Field.oneof_index" + Field_Packed_field_fullname protoreflect.FullName = "google.protobuf.Field.packed" + Field_Options_field_fullname protoreflect.FullName = "google.protobuf.Field.options" + Field_JsonName_field_fullname protoreflect.FullName = "google.protobuf.Field.json_name" + Field_DefaultValue_field_fullname protoreflect.FullName = "google.protobuf.Field.default_value" +) + +// Field numbers for google.protobuf.Field. +const ( + Field_Kind_field_number protoreflect.FieldNumber = 1 + Field_Cardinality_field_number protoreflect.FieldNumber = 2 + Field_Number_field_number protoreflect.FieldNumber = 3 + Field_Name_field_number protoreflect.FieldNumber = 4 + Field_TypeUrl_field_number protoreflect.FieldNumber = 6 + Field_OneofIndex_field_number protoreflect.FieldNumber = 7 + Field_Packed_field_number protoreflect.FieldNumber = 8 + Field_Options_field_number protoreflect.FieldNumber = 9 + Field_JsonName_field_number protoreflect.FieldNumber = 10 + Field_DefaultValue_field_number protoreflect.FieldNumber = 11 +) + +// Full and short names for google.protobuf.Field.Kind. +const ( + Field_Kind_enum_fullname = "google.protobuf.Field.Kind" + Field_Kind_enum_name = "Kind" +) + +// Full and short names for google.protobuf.Field.Cardinality. +const ( + Field_Cardinality_enum_fullname = "google.protobuf.Field.Cardinality" + Field_Cardinality_enum_name = "Cardinality" +) + +// Names for google.protobuf.Enum. +const ( + Enum_message_name protoreflect.Name = "Enum" + Enum_message_fullname protoreflect.FullName = "google.protobuf.Enum" +) + +// Field names for google.protobuf.Enum. +const ( + Enum_Name_field_name protoreflect.Name = "name" + Enum_Enumvalue_field_name protoreflect.Name = "enumvalue" + Enum_Options_field_name protoreflect.Name = "options" + Enum_SourceContext_field_name protoreflect.Name = "source_context" + Enum_Syntax_field_name protoreflect.Name = "syntax" + Enum_Edition_field_name protoreflect.Name = "edition" + + Enum_Name_field_fullname protoreflect.FullName = "google.protobuf.Enum.name" + Enum_Enumvalue_field_fullname protoreflect.FullName = "google.protobuf.Enum.enumvalue" + Enum_Options_field_fullname protoreflect.FullName = "google.protobuf.Enum.options" + Enum_SourceContext_field_fullname protoreflect.FullName = "google.protobuf.Enum.source_context" + Enum_Syntax_field_fullname protoreflect.FullName = "google.protobuf.Enum.syntax" + Enum_Edition_field_fullname protoreflect.FullName = "google.protobuf.Enum.edition" +) + +// Field numbers for google.protobuf.Enum. +const ( + Enum_Name_field_number protoreflect.FieldNumber = 1 + Enum_Enumvalue_field_number protoreflect.FieldNumber = 2 + Enum_Options_field_number protoreflect.FieldNumber = 3 + Enum_SourceContext_field_number protoreflect.FieldNumber = 4 + Enum_Syntax_field_number protoreflect.FieldNumber = 5 + Enum_Edition_field_number protoreflect.FieldNumber = 6 +) + +// Names for google.protobuf.EnumValue. +const ( + EnumValue_message_name protoreflect.Name = "EnumValue" + EnumValue_message_fullname protoreflect.FullName = "google.protobuf.EnumValue" +) + +// Field names for google.protobuf.EnumValue. +const ( + EnumValue_Name_field_name protoreflect.Name = "name" + EnumValue_Number_field_name protoreflect.Name = "number" + EnumValue_Options_field_name protoreflect.Name = "options" + + EnumValue_Name_field_fullname protoreflect.FullName = "google.protobuf.EnumValue.name" + EnumValue_Number_field_fullname protoreflect.FullName = "google.protobuf.EnumValue.number" + EnumValue_Options_field_fullname protoreflect.FullName = "google.protobuf.EnumValue.options" +) + +// Field numbers for google.protobuf.EnumValue. +const ( + EnumValue_Name_field_number protoreflect.FieldNumber = 1 + EnumValue_Number_field_number protoreflect.FieldNumber = 2 + EnumValue_Options_field_number protoreflect.FieldNumber = 3 +) + +// Names for google.protobuf.Option. +const ( + Option_message_name protoreflect.Name = "Option" + Option_message_fullname protoreflect.FullName = "google.protobuf.Option" +) + +// Field names for google.protobuf.Option. +const ( + Option_Name_field_name protoreflect.Name = "name" + Option_Value_field_name protoreflect.Name = "value" + + Option_Name_field_fullname protoreflect.FullName = "google.protobuf.Option.name" + Option_Value_field_fullname protoreflect.FullName = "google.protobuf.Option.value" +) + +// Field numbers for google.protobuf.Option. +const ( + Option_Name_field_number protoreflect.FieldNumber = 1 + Option_Value_field_number protoreflect.FieldNumber = 2 +) diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/wrappers.go b/vendor/go.temporal.io/api/internal/protojson/genid/wrappers.go new file mode 100644 index 00000000000..429384b85b0 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/wrappers.go @@ -0,0 +1,13 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package genid + +import protoreflect "google.golang.org/protobuf/reflect/protoreflect" + +// Generic field name and number for messages in wrappers.proto. +const ( + WrapperValue_Value_field_name protoreflect.Name = "value" + WrapperValue_Value_field_number protoreflect.FieldNumber = 1 +) diff --git a/vendor/go.temporal.io/api/internal/protojson/genid/wrappers_gen.go b/vendor/go.temporal.io/api/internal/protojson/genid/wrappers_gen.go new file mode 100644 index 00000000000..72527d2ab03 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/genid/wrappers_gen.go @@ -0,0 +1,175 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Code generated by generate-protos. DO NOT EDIT. + +package genid + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" +) + +const File_google_protobuf_wrappers_proto = "google/protobuf/wrappers.proto" + +// Names for google.protobuf.DoubleValue. +const ( + DoubleValue_message_name protoreflect.Name = "DoubleValue" + DoubleValue_message_fullname protoreflect.FullName = "google.protobuf.DoubleValue" +) + +// Field names for google.protobuf.DoubleValue. +const ( + DoubleValue_Value_field_name protoreflect.Name = "value" + + DoubleValue_Value_field_fullname protoreflect.FullName = "google.protobuf.DoubleValue.value" +) + +// Field numbers for google.protobuf.DoubleValue. +const ( + DoubleValue_Value_field_number protoreflect.FieldNumber = 1 +) + +// Names for google.protobuf.FloatValue. +const ( + FloatValue_message_name protoreflect.Name = "FloatValue" + FloatValue_message_fullname protoreflect.FullName = "google.protobuf.FloatValue" +) + +// Field names for google.protobuf.FloatValue. +const ( + FloatValue_Value_field_name protoreflect.Name = "value" + + FloatValue_Value_field_fullname protoreflect.FullName = "google.protobuf.FloatValue.value" +) + +// Field numbers for google.protobuf.FloatValue. +const ( + FloatValue_Value_field_number protoreflect.FieldNumber = 1 +) + +// Names for google.protobuf.Int64Value. +const ( + Int64Value_message_name protoreflect.Name = "Int64Value" + Int64Value_message_fullname protoreflect.FullName = "google.protobuf.Int64Value" +) + +// Field names for google.protobuf.Int64Value. +const ( + Int64Value_Value_field_name protoreflect.Name = "value" + + Int64Value_Value_field_fullname protoreflect.FullName = "google.protobuf.Int64Value.value" +) + +// Field numbers for google.protobuf.Int64Value. +const ( + Int64Value_Value_field_number protoreflect.FieldNumber = 1 +) + +// Names for google.protobuf.UInt64Value. +const ( + UInt64Value_message_name protoreflect.Name = "UInt64Value" + UInt64Value_message_fullname protoreflect.FullName = "google.protobuf.UInt64Value" +) + +// Field names for google.protobuf.UInt64Value. +const ( + UInt64Value_Value_field_name protoreflect.Name = "value" + + UInt64Value_Value_field_fullname protoreflect.FullName = "google.protobuf.UInt64Value.value" +) + +// Field numbers for google.protobuf.UInt64Value. +const ( + UInt64Value_Value_field_number protoreflect.FieldNumber = 1 +) + +// Names for google.protobuf.Int32Value. +const ( + Int32Value_message_name protoreflect.Name = "Int32Value" + Int32Value_message_fullname protoreflect.FullName = "google.protobuf.Int32Value" +) + +// Field names for google.protobuf.Int32Value. +const ( + Int32Value_Value_field_name protoreflect.Name = "value" + + Int32Value_Value_field_fullname protoreflect.FullName = "google.protobuf.Int32Value.value" +) + +// Field numbers for google.protobuf.Int32Value. +const ( + Int32Value_Value_field_number protoreflect.FieldNumber = 1 +) + +// Names for google.protobuf.UInt32Value. +const ( + UInt32Value_message_name protoreflect.Name = "UInt32Value" + UInt32Value_message_fullname protoreflect.FullName = "google.protobuf.UInt32Value" +) + +// Field names for google.protobuf.UInt32Value. +const ( + UInt32Value_Value_field_name protoreflect.Name = "value" + + UInt32Value_Value_field_fullname protoreflect.FullName = "google.protobuf.UInt32Value.value" +) + +// Field numbers for google.protobuf.UInt32Value. +const ( + UInt32Value_Value_field_number protoreflect.FieldNumber = 1 +) + +// Names for google.protobuf.BoolValue. +const ( + BoolValue_message_name protoreflect.Name = "BoolValue" + BoolValue_message_fullname protoreflect.FullName = "google.protobuf.BoolValue" +) + +// Field names for google.protobuf.BoolValue. +const ( + BoolValue_Value_field_name protoreflect.Name = "value" + + BoolValue_Value_field_fullname protoreflect.FullName = "google.protobuf.BoolValue.value" +) + +// Field numbers for google.protobuf.BoolValue. +const ( + BoolValue_Value_field_number protoreflect.FieldNumber = 1 +) + +// Names for google.protobuf.StringValue. +const ( + StringValue_message_name protoreflect.Name = "StringValue" + StringValue_message_fullname protoreflect.FullName = "google.protobuf.StringValue" +) + +// Field names for google.protobuf.StringValue. +const ( + StringValue_Value_field_name protoreflect.Name = "value" + + StringValue_Value_field_fullname protoreflect.FullName = "google.protobuf.StringValue.value" +) + +// Field numbers for google.protobuf.StringValue. +const ( + StringValue_Value_field_number protoreflect.FieldNumber = 1 +) + +// Names for google.protobuf.BytesValue. +const ( + BytesValue_message_name protoreflect.Name = "BytesValue" + BytesValue_message_fullname protoreflect.FullName = "google.protobuf.BytesValue" +) + +// Field names for google.protobuf.BytesValue. +const ( + BytesValue_Value_field_name protoreflect.Name = "value" + + BytesValue_Value_field_fullname protoreflect.FullName = "google.protobuf.BytesValue.value" +) + +// Field numbers for google.protobuf.BytesValue. +const ( + BytesValue_Value_field_number protoreflect.FieldNumber = 1 +) diff --git a/vendor/go.temporal.io/api/internal/protojson/json/decode.go b/vendor/go.temporal.io/api/internal/protojson/json/decode.go new file mode 100644 index 00000000000..cc791e42da2 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/json/decode.go @@ -0,0 +1,340 @@ +// Copyright 2018 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package json + +import ( + "bytes" + "fmt" + "io" + "regexp" + "unicode/utf8" + + "go.temporal.io/api/internal/protojson/errors" +) + +// call specifies which Decoder method was invoked. +type call uint8 + +const ( + readCall call = iota + peekCall +) + +const unexpectedFmt = "unexpected token %s" + +// ErrUnexpectedEOF means that EOF was encountered in the middle of the input. +var ErrUnexpectedEOF = errors.New("%v", io.ErrUnexpectedEOF) + +// Decoder is a token-based JSON decoder. +type Decoder struct { + // lastCall is last method called, either readCall or peekCall. + // Initial value is readCall. + lastCall call + + // lastToken contains the last read token. + lastToken Token + + // lastErr contains the last read error. + lastErr error + + // openStack is a stack containing ObjectOpen and ArrayOpen values. The + // top of stack represents the object or the array the current value is + // directly located in. + openStack []Kind + + // orig is used in reporting line and column. + orig []byte + // in contains the unconsumed input. + in []byte +} + +// NewDecoder returns a Decoder to read the given []byte. +func NewDecoder(b []byte) *Decoder { + return &Decoder{orig: b, in: b} +} + +// Peek looks ahead and returns the next token kind without advancing a read. +func (d *Decoder) Peek() (Token, error) { + defer func() { d.lastCall = peekCall }() + if d.lastCall == readCall { + d.lastToken, d.lastErr = d.Read() + } + return d.lastToken, d.lastErr +} + +// Read returns the next JSON token. +// It will return an error if there is no valid token. +func (d *Decoder) Read() (Token, error) { + const scalar = Null | Bool | Number | String + + defer func() { d.lastCall = readCall }() + if d.lastCall == peekCall { + return d.lastToken, d.lastErr + } + + tok, err := d.parseNext() + if err != nil { + return Token{}, err + } + + switch tok.kind { + case EOF: + if len(d.openStack) != 0 || + d.lastToken.kind&scalar|ObjectClose|ArrayClose == 0 { + return Token{}, ErrUnexpectedEOF + } + + case Null: + if !d.isValueNext() { + return Token{}, d.newSyntaxError(tok.pos, unexpectedFmt, tok.RawString()) + } + + case Bool, Number: + if !d.isValueNext() { + return Token{}, d.newSyntaxError(tok.pos, unexpectedFmt, tok.RawString()) + } + + case String: + if d.isValueNext() { + break + } + // This string token should only be for a field name. + if d.lastToken.kind&(ObjectOpen|comma) == 0 { + return Token{}, d.newSyntaxError(tok.pos, unexpectedFmt, tok.RawString()) + } + if len(d.in) == 0 { + return Token{}, ErrUnexpectedEOF + } + if c := d.in[0]; c != ':' { + return Token{}, d.newSyntaxError(d.currPos(), `unexpected character %s, missing ":" after field name`, string(c)) + } + tok.kind = Name + d.consume(1) + + case ObjectOpen, ArrayOpen: + if !d.isValueNext() { + return Token{}, d.newSyntaxError(tok.pos, unexpectedFmt, tok.RawString()) + } + d.openStack = append(d.openStack, tok.kind) + + case ObjectClose: + if len(d.openStack) == 0 || + d.lastToken.kind&(Name|comma) != 0 || + d.openStack[len(d.openStack)-1] != ObjectOpen { + return Token{}, d.newSyntaxError(tok.pos, unexpectedFmt, tok.RawString()) + } + d.openStack = d.openStack[:len(d.openStack)-1] + + case ArrayClose: + if len(d.openStack) == 0 || + d.lastToken.kind == comma || + d.openStack[len(d.openStack)-1] != ArrayOpen { + return Token{}, d.newSyntaxError(tok.pos, unexpectedFmt, tok.RawString()) + } + d.openStack = d.openStack[:len(d.openStack)-1] + + case comma: + if len(d.openStack) == 0 || + d.lastToken.kind&(scalar|ObjectClose|ArrayClose) == 0 { + return Token{}, d.newSyntaxError(tok.pos, unexpectedFmt, tok.RawString()) + } + } + + // Update d.lastToken only after validating token to be in the right sequence. + d.lastToken = tok + + if d.lastToken.kind == comma { + return d.Read() + } + return tok, nil +} + +// Any sequence that looks like a non-delimiter (for error reporting). +var errRegexp = regexp.MustCompile(`^([-+._a-zA-Z0-9]{1,32}|.)`) + +// parseNext parses for the next JSON token. It returns a Token object for +// different types, except for Name. It does not handle whether the next token +// is in a valid sequence or not. +func (d *Decoder) parseNext() (Token, error) { + // Trim leading spaces. + d.consume(0) + + in := d.in + if len(in) == 0 { + return d.consumeToken(EOF, 0), nil + } + + switch in[0] { + case 'n': + if n := matchWithDelim("null", in); n != 0 { + return d.consumeToken(Null, n), nil + } + + case 't': + if n := matchWithDelim("true", in); n != 0 { + return d.consumeBoolToken(true, n), nil + } + + case 'f': + if n := matchWithDelim("false", in); n != 0 { + return d.consumeBoolToken(false, n), nil + } + + case '-', '0', '1', '2', '3', '4', '5', '6', '7', '8', '9': + if n, ok := parseNumber(in); ok { + return d.consumeToken(Number, n), nil + } + + case '"': + s, n, err := d.parseString(in) + if err != nil { + return Token{}, err + } + return d.consumeStringToken(s, n), nil + + case '{': + return d.consumeToken(ObjectOpen, 1), nil + + case '}': + return d.consumeToken(ObjectClose, 1), nil + + case '[': + return d.consumeToken(ArrayOpen, 1), nil + + case ']': + return d.consumeToken(ArrayClose, 1), nil + + case ',': + return d.consumeToken(comma, 1), nil + } + return Token{}, d.newSyntaxError(d.currPos(), "invalid value %s", errRegexp.Find(in)) +} + +// newSyntaxError returns an error with line and column information useful for +// syntax errors. +func (d *Decoder) newSyntaxError(pos int, f string, x ...interface{}) error { + e := errors.New(f, x...) + line, column := d.Position(pos) + return errors.New("syntax error (line %d:%d): %v", line, column, e) +} + +// Position returns line and column number of given index of the original input. +// It will panic if index is out of range. +func (d *Decoder) Position(idx int) (line int, column int) { + b := d.orig[:idx] + line = bytes.Count(b, []byte("\n")) + 1 + if i := bytes.LastIndexByte(b, '\n'); i >= 0 { + b = b[i+1:] + } + column = utf8.RuneCount(b) + 1 // ignore multi-rune characters + return line, column +} + +// currPos returns the current index position of d.in from d.orig. +func (d *Decoder) currPos() int { + return len(d.orig) - len(d.in) +} + +// matchWithDelim matches s with the input b and verifies that the match +// terminates with a delimiter of some form (e.g., r"[^-+_.a-zA-Z0-9]"). +// As a special case, EOF is considered a delimiter. It returns the length of s +// if there is a match, else 0. +func matchWithDelim(s string, b []byte) int { + if !bytes.HasPrefix(b, []byte(s)) { + return 0 + } + + n := len(s) + if n < len(b) && isNotDelim(b[n]) { + return 0 + } + return n +} + +// isNotDelim returns true if given byte is a not delimiter character. +func isNotDelim(c byte) bool { + return (c == '-' || c == '+' || c == '.' || c == '_' || + ('a' <= c && c <= 'z') || + ('A' <= c && c <= 'Z') || + ('0' <= c && c <= '9')) +} + +// consume consumes n bytes of input and any subsequent whitespace. +func (d *Decoder) consume(n int) { + d.in = d.in[n:] + for len(d.in) > 0 { + switch d.in[0] { + case ' ', '\n', '\r', '\t': + d.in = d.in[1:] + default: + return + } + } +} + +// isValueNext returns true if next type should be a JSON value: Null, +// Number, String or Bool. +func (d *Decoder) isValueNext() bool { + if len(d.openStack) == 0 { + return d.lastToken.kind == 0 + } + + start := d.openStack[len(d.openStack)-1] + switch start { + case ObjectOpen: + return d.lastToken.kind&Name != 0 + case ArrayOpen: + return d.lastToken.kind&(ArrayOpen|comma) != 0 + } + panic(fmt.Sprintf( + "unreachable logic in Decoder.isValueNext, lastToken.kind: %v, openStack: %v", + d.lastToken.kind, start)) +} + +// consumeToken constructs a Token for given Kind with raw value derived from +// current d.in and given size, and consumes the given size-length of it. +func (d *Decoder) consumeToken(kind Kind, size int) Token { + tok := Token{ + kind: kind, + raw: d.in[:size], + pos: len(d.orig) - len(d.in), + } + d.consume(size) + return tok +} + +// consumeBoolToken constructs a Token for a Bool kind with raw value derived from +// current d.in and given size. +func (d *Decoder) consumeBoolToken(b bool, size int) Token { + tok := Token{ + kind: Bool, + raw: d.in[:size], + pos: len(d.orig) - len(d.in), + boo: b, + } + d.consume(size) + return tok +} + +// consumeStringToken constructs a Token for a String kind with raw value derived +// from current d.in and given size. +func (d *Decoder) consumeStringToken(s string, size int) Token { + tok := Token{ + kind: String, + raw: d.in[:size], + pos: len(d.orig) - len(d.in), + str: s, + } + d.consume(size) + return tok +} + +// Clone returns a copy of the Decoder for use in reading ahead the next JSON +// object, array or other values without affecting current Decoder. +func (d *Decoder) Clone() *Decoder { + ret := *d + ret.openStack = append([]Kind(nil), ret.openStack...) + return &ret +} diff --git a/vendor/go.temporal.io/api/internal/protojson/json/decode_number.go b/vendor/go.temporal.io/api/internal/protojson/json/decode_number.go new file mode 100644 index 00000000000..2999d713320 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/json/decode_number.go @@ -0,0 +1,254 @@ +// Copyright 2018 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package json + +import ( + "bytes" + "strconv" +) + +// parseNumber reads the given []byte for a valid JSON number. If it is valid, +// it returns the number of bytes. Parsing logic follows the definition in +// https://tools.ietf.org/html/rfc7159#section-6, and is based off +// encoding/json.isValidNumber function. +func parseNumber(input []byte) (int, bool) { + var n int + + s := input + if len(s) == 0 { + return 0, false + } + + // Optional - + if s[0] == '-' { + s = s[1:] + n++ + if len(s) == 0 { + return 0, false + } + } + + // Digits + switch { + case s[0] == '0': + s = s[1:] + n++ + + case '1' <= s[0] && s[0] <= '9': + s = s[1:] + n++ + for len(s) > 0 && '0' <= s[0] && s[0] <= '9' { + s = s[1:] + n++ + } + + default: + return 0, false + } + + // . followed by 1 or more digits. + if len(s) >= 2 && s[0] == '.' && '0' <= s[1] && s[1] <= '9' { + s = s[2:] + n += 2 + for len(s) > 0 && '0' <= s[0] && s[0] <= '9' { + s = s[1:] + n++ + } + } + + // e or E followed by an optional - or + and + // 1 or more digits. + if len(s) >= 2 && (s[0] == 'e' || s[0] == 'E') { + s = s[1:] + n++ + if s[0] == '+' || s[0] == '-' { + s = s[1:] + n++ + if len(s) == 0 { + return 0, false + } + } + for len(s) > 0 && '0' <= s[0] && s[0] <= '9' { + s = s[1:] + n++ + } + } + + // Check that next byte is a delimiter or it is at the end. + if n < len(input) && isNotDelim(input[n]) { + return 0, false + } + + return n, true +} + +// numberParts is the result of parsing out a valid JSON number. It contains +// the parts of a number. The parts are used for integer conversion. +type numberParts struct { + neg bool + intp []byte + frac []byte + exp []byte +} + +// parseNumber constructs numberParts from given []byte. The logic here is +// similar to consumeNumber above with the difference of having to construct +// numberParts. The slice fields in numberParts are subslices of the input. +func parseNumberParts(input []byte) (numberParts, bool) { + var neg bool + var intp []byte + var frac []byte + var exp []byte + + s := input + if len(s) == 0 { + return numberParts{}, false + } + + // Optional - + if s[0] == '-' { + neg = true + s = s[1:] + if len(s) == 0 { + return numberParts{}, false + } + } + + // Digits + switch { + case s[0] == '0': + // Skip first 0 and no need to store. + s = s[1:] + + case '1' <= s[0] && s[0] <= '9': + intp = s + n := 1 + s = s[1:] + for len(s) > 0 && '0' <= s[0] && s[0] <= '9' { + s = s[1:] + n++ + } + intp = intp[:n] + + default: + return numberParts{}, false + } + + // . followed by 1 or more digits. + if len(s) >= 2 && s[0] == '.' && '0' <= s[1] && s[1] <= '9' { + frac = s[1:] + n := 1 + s = s[2:] + for len(s) > 0 && '0' <= s[0] && s[0] <= '9' { + s = s[1:] + n++ + } + frac = frac[:n] + } + + // e or E followed by an optional - or + and + // 1 or more digits. + if len(s) >= 2 && (s[0] == 'e' || s[0] == 'E') { + s = s[1:] + exp = s + n := 0 + if s[0] == '+' || s[0] == '-' { + s = s[1:] + n++ + if len(s) == 0 { + return numberParts{}, false + } + } + for len(s) > 0 && '0' <= s[0] && s[0] <= '9' { + s = s[1:] + n++ + } + exp = exp[:n] + } + + return numberParts{ + neg: neg, + intp: intp, + frac: bytes.TrimRight(frac, "0"), // Remove unnecessary 0s to the right. + exp: exp, + }, true +} + +// normalizeToIntString returns an integer string in normal form without the +// E-notation for given numberParts. It will return false if it is not an +// integer or if the exponent exceeds than max/min int value. +func normalizeToIntString(n numberParts) (string, bool) { + intpSize := len(n.intp) + fracSize := len(n.frac) + + if intpSize == 0 && fracSize == 0 { + return "0", true + } + + var exp int + if len(n.exp) > 0 { + i, err := strconv.ParseInt(string(n.exp), 10, 32) + if err != nil { + return "", false + } + exp = int(i) + } + + var num []byte + if exp >= 0 { + // For positive E, shift fraction digits into integer part and also pad + // with zeroes as needed. + + // If there are more digits in fraction than the E value, then the + // number is not an integer. + if fracSize > exp { + return "", false + } + + // Make sure resulting digits are within max value limit to avoid + // unnecessarily constructing a large byte slice that may simply fail + // later on. + const maxDigits = 20 // Max uint64 value has 20 decimal digits. + if intpSize+exp > maxDigits { + return "", false + } + + // Set cap to make a copy of integer part when appended. + num = n.intp[:len(n.intp):len(n.intp)] + num = append(num, n.frac...) + for i := 0; i < exp-fracSize; i++ { + num = append(num, '0') + } + } else { + // For negative E, shift digits in integer part out. + + // If there are fractions, then the number is not an integer. + if fracSize > 0 { + return "", false + } + + // index is where the decimal point will be after adjusting for negative + // exponent. + index := intpSize + exp + if index < 0 { + return "", false + } + + num = n.intp + // If any of the digits being shifted to the right of the decimal point + // is non-zero, then the number is not an integer. + for i := index; i < intpSize; i++ { + if num[i] != '0' { + return "", false + } + } + num = num[:index] + } + + if n.neg { + return "-" + string(num), true + } + return string(num), true +} diff --git a/vendor/go.temporal.io/api/internal/protojson/json/decode_string.go b/vendor/go.temporal.io/api/internal/protojson/json/decode_string.go new file mode 100644 index 00000000000..4a2b745ca63 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/json/decode_string.go @@ -0,0 +1,91 @@ +// Copyright 2018 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package json + +import ( + "strconv" + "unicode" + "unicode/utf16" + "unicode/utf8" + + "go.temporal.io/api/internal/protojson/strs" +) + +func (d *Decoder) parseString(in []byte) (string, int, error) { + in0 := in + if len(in) == 0 { + return "", 0, ErrUnexpectedEOF + } + if in[0] != '"' { + return "", 0, d.newSyntaxError(d.currPos(), "invalid character %q at start of string", in[0]) + } + in = in[1:] + i := indexNeedEscapeInBytes(in) + in, out := in[i:], in[:i:i] // set cap to prevent mutations + for len(in) > 0 { + switch r, n := utf8.DecodeRune(in); { + case r == utf8.RuneError && n == 1: + return "", 0, d.newSyntaxError(d.currPos(), "invalid UTF-8 in string") + case r < ' ': + return "", 0, d.newSyntaxError(d.currPos(), "invalid character %q in string", r) + case r == '"': + in = in[1:] + n := len(in0) - len(in) + return string(out), n, nil + case r == '\\': + if len(in) < 2 { + return "", 0, ErrUnexpectedEOF + } + switch r := in[1]; r { + case '"', '\\', '/': + in, out = in[2:], append(out, r) + case 'b': + in, out = in[2:], append(out, '\b') + case 'f': + in, out = in[2:], append(out, '\f') + case 'n': + in, out = in[2:], append(out, '\n') + case 'r': + in, out = in[2:], append(out, '\r') + case 't': + in, out = in[2:], append(out, '\t') + case 'u': + if len(in) < 6 { + return "", 0, ErrUnexpectedEOF + } + v, err := strconv.ParseUint(string(in[2:6]), 16, 16) + if err != nil { + return "", 0, d.newSyntaxError(d.currPos(), "invalid escape code %q in string", in[:6]) + } + in = in[6:] + + r := rune(v) + if utf16.IsSurrogate(r) { + if len(in) < 6 { + return "", 0, ErrUnexpectedEOF + } + v, err := strconv.ParseUint(string(in[2:6]), 16, 16) + r = utf16.DecodeRune(r, rune(v)) + if in[0] != '\\' || in[1] != 'u' || + r == unicode.ReplacementChar || err != nil { + return "", 0, d.newSyntaxError(d.currPos(), "invalid escape code %q in string", in[:6]) + } + in = in[6:] + } + out = append(out, string(r)...) + default: + return "", 0, d.newSyntaxError(d.currPos(), "invalid escape code %q in string", in[:2]) + } + default: + i := indexNeedEscapeInBytes(in[n:]) + in, out = in[n+i:], append(out, in[:n+i]...) + } + } + return "", 0, ErrUnexpectedEOF +} + +// indexNeedEscapeInBytes returns the index of the character that needs +// escaping. If no characters need escaping, this returns the input length. +func indexNeedEscapeInBytes(b []byte) int { return indexNeedEscapeInString(strs.UnsafeString(b)) } diff --git a/vendor/go.temporal.io/api/internal/protojson/json/decode_token.go b/vendor/go.temporal.io/api/internal/protojson/json/decode_token.go new file mode 100644 index 00000000000..50578d6593c --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/json/decode_token.go @@ -0,0 +1,192 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package json + +import ( + "bytes" + "fmt" + "strconv" +) + +// Kind represents a token kind expressible in the JSON format. +type Kind uint16 + +const ( + Invalid Kind = (1 << iota) / 2 + EOF + Null + Bool + Number + String + Name + ObjectOpen + ObjectClose + ArrayOpen + ArrayClose + + // comma is only for parsing in between tokens and + // does not need to be exported. + comma +) + +func (k Kind) String() string { + switch k { + case EOF: + return "eof" + case Null: + return "null" + case Bool: + return "bool" + case Number: + return "number" + case String: + return "string" + case ObjectOpen: + return "{" + case ObjectClose: + return "}" + case Name: + return "name" + case ArrayOpen: + return "[" + case ArrayClose: + return "]" + case comma: + return "," + } + return "" +} + +// Token provides a parsed token kind and value. +// +// Values are provided by the difference accessor methods. The accessor methods +// Name, Bool, and ParsedString will panic if called on the wrong kind. There +// are different accessor methods for the Number kind for converting to the +// appropriate Go numeric type and those methods have the ok return value. +type Token struct { + // Token kind. + kind Kind + // pos provides the position of the token in the original input. + pos int + // raw bytes of the serialized token. + // This is a subslice into the original input. + raw []byte + // boo is parsed boolean value. + boo bool + // str is parsed string value. + str string +} + +// Kind returns the token kind. +func (t Token) Kind() Kind { + return t.kind +} + +// RawString returns the read value in string. +func (t Token) RawString() string { + return string(t.raw) +} + +// Pos returns the token position from the input. +func (t Token) Pos() int { + return t.pos +} + +// Name returns the object name if token is Name, else it panics. +func (t Token) Name() string { + if t.kind == Name { + return t.str + } + panic(fmt.Sprintf("Token is not a Name: %v", t.RawString())) +} + +// Bool returns the bool value if token kind is Bool, else it panics. +func (t Token) Bool() bool { + if t.kind == Bool { + return t.boo + } + panic(fmt.Sprintf("Token is not a Bool: %v", t.RawString())) +} + +// ParsedString returns the string value for a JSON string token or the read +// value in string if token is not a string. +func (t Token) ParsedString() string { + if t.kind == String { + return t.str + } + panic(fmt.Sprintf("Token is not a String: %v", t.RawString())) +} + +// Float returns the floating-point number if token kind is Number. +// +// The floating-point precision is specified by the bitSize parameter: 32 for +// float32 or 64 for float64. If bitSize=32, the result still has type float64, +// but it will be convertible to float32 without changing its value. It will +// return false if the number exceeds the floating point limits for given +// bitSize. +func (t Token) Float(bitSize int) (float64, bool) { + if t.kind != Number { + return 0, false + } + f, err := strconv.ParseFloat(t.RawString(), bitSize) + if err != nil { + return 0, false + } + return f, true +} + +// Int returns the signed integer number if token is Number. +// +// The given bitSize specifies the integer type that the result must fit into. +// It returns false if the number is not an integer value or if the result +// exceeds the limits for given bitSize. +func (t Token) Int(bitSize int) (int64, bool) { + s, ok := t.getIntStr() + if !ok { + return 0, false + } + n, err := strconv.ParseInt(s, 10, bitSize) + if err != nil { + return 0, false + } + return n, true +} + +// Uint returns the signed integer number if token is Number. +// +// The given bitSize specifies the unsigned integer type that the result must +// fit into. It returns false if the number is not an unsigned integer value +// or if the result exceeds the limits for given bitSize. +func (t Token) Uint(bitSize int) (uint64, bool) { + s, ok := t.getIntStr() + if !ok { + return 0, false + } + n, err := strconv.ParseUint(s, 10, bitSize) + if err != nil { + return 0, false + } + return n, true +} + +func (t Token) getIntStr() (string, bool) { + if t.kind != Number { + return "", false + } + parts, ok := parseNumberParts(t.raw) + if !ok { + return "", false + } + return normalizeToIntString(parts) +} + +// TokenEquals returns true if given Tokens are equal, else false. +func TokenEquals(x, y Token) bool { + return x.kind == y.kind && + x.pos == y.pos && + bytes.Equal(x.raw, y.raw) && + x.boo == y.boo && + x.str == y.str +} diff --git a/vendor/go.temporal.io/api/internal/protojson/json/encode.go b/vendor/go.temporal.io/api/internal/protojson/json/encode.go new file mode 100644 index 00000000000..5da62beaccf --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/json/encode.go @@ -0,0 +1,267 @@ +// Copyright 2018 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package json + +import ( + "math" + "math/bits" + "strconv" + "strings" + "unicode/utf8" + + "go.temporal.io/api/internal/protojson/errors" +) + +// kind represents an encoding type. +type kind uint8 + +const ( + _ kind = (1 << iota) / 2 + name + scalar + objectOpen + objectClose + arrayOpen + arrayClose +) + +// Encoder provides methods to write out JSON constructs and values. The user is +// responsible for producing valid sequences of JSON constructs and values. +type Encoder struct { + indent string + lastKind kind + indents []byte + out []byte +} + +// NewEncoder returns an Encoder. +// +// If indent is a non-empty string, it causes every entry for an Array or Object +// to be preceded by the indent and trailed by a newline. +func NewEncoder(buf []byte, indent string) (*Encoder, error) { + e := &Encoder{ + out: buf, + } + if len(indent) > 0 { + if strings.Trim(indent, " \t") != "" { + return nil, errors.New("indent may only be composed of space or tab characters") + } + e.indent = indent + } + return e, nil +} + +// Bytes returns the content of the written bytes. +func (e *Encoder) Bytes() []byte { + return e.out +} + +// WriteNull writes out the null value. +func (e *Encoder) WriteNull() { + e.prepareNext(scalar) + e.out = append(e.out, "null"...) +} + +// WriteBool writes out the given boolean value. +func (e *Encoder) WriteBool(b bool) { + e.prepareNext(scalar) + if b { + e.out = append(e.out, "true"...) + } else { + e.out = append(e.out, "false"...) + } +} + +// WriteString writes out the given string in JSON string value. Returns error +// if input string contains invalid UTF-8. +func (e *Encoder) WriteString(s string) error { + e.prepareNext(scalar) + var err error + if e.out, err = appendString(e.out, s); err != nil { + return err + } + return nil +} + +// Sentinel error used for indicating invalid UTF-8. +var errInvalidUTF8 = errors.New("invalid UTF-8") + +func appendString(out []byte, in string) ([]byte, error) { + out = append(out, '"') + i := indexNeedEscapeInString(in) + in, out = in[i:], append(out, in[:i]...) + for len(in) > 0 { + switch r, n := utf8.DecodeRuneInString(in); { + case r == utf8.RuneError && n == 1: + return out, errInvalidUTF8 + case r < ' ' || r == '"' || r == '\\': + out = append(out, '\\') + switch r { + case '"', '\\': + out = append(out, byte(r)) + case '\b': + out = append(out, 'b') + case '\f': + out = append(out, 'f') + case '\n': + out = append(out, 'n') + case '\r': + out = append(out, 'r') + case '\t': + out = append(out, 't') + default: + out = append(out, 'u') + out = append(out, "0000"[1+(bits.Len32(uint32(r))-1)/4:]...) + out = strconv.AppendUint(out, uint64(r), 16) + } + in = in[n:] + default: + i := indexNeedEscapeInString(in[n:]) + in, out = in[n+i:], append(out, in[:n+i]...) + } + } + out = append(out, '"') + return out, nil +} + +// indexNeedEscapeInString returns the index of the character that needs +// escaping. If no characters need escaping, this returns the input length. +func indexNeedEscapeInString(s string) int { + for i, r := range s { + if r < ' ' || r == '\\' || r == '"' || r == utf8.RuneError { + return i + } + } + return len(s) +} + +// WriteFloat writes out the given float and bitSize in JSON number value. +func (e *Encoder) WriteFloat(n float64, bitSize int) { + e.prepareNext(scalar) + e.out = appendFloat(e.out, n, bitSize) +} + +// appendFloat formats given float in bitSize, and appends to the given []byte. +func appendFloat(out []byte, n float64, bitSize int) []byte { + switch { + case math.IsNaN(n): + return append(out, `"NaN"`...) + case math.IsInf(n, +1): + return append(out, `"Infinity"`...) + case math.IsInf(n, -1): + return append(out, `"-Infinity"`...) + } + + // JSON number formatting logic based on encoding/json. + // See floatEncoder.encode for reference. + fmt := byte('f') + if abs := math.Abs(n); abs != 0 { + if bitSize == 64 && (abs < 1e-6 || abs >= 1e21) || + bitSize == 32 && (float32(abs) < 1e-6 || float32(abs) >= 1e21) { + fmt = 'e' + } + } + out = strconv.AppendFloat(out, n, fmt, -1, bitSize) + if fmt == 'e' { + n := len(out) + if n >= 4 && out[n-4] == 'e' && out[n-3] == '-' && out[n-2] == '0' { + out[n-2] = out[n-1] + out = out[:n-1] + } + } + return out +} + +// WriteInt writes out the given signed integer in JSON number value. +func (e *Encoder) WriteInt(n int64) { + e.prepareNext(scalar) + e.out = strconv.AppendInt(e.out, n, 10) +} + +// WriteUint writes out the given unsigned integer in JSON number value. +func (e *Encoder) WriteUint(n uint64) { + e.prepareNext(scalar) + e.out = strconv.AppendUint(e.out, n, 10) +} + +// StartObject writes out the '{' symbol. +func (e *Encoder) StartObject() { + e.prepareNext(objectOpen) + e.out = append(e.out, '{') +} + +// EndObject writes out the '}' symbol. +func (e *Encoder) EndObject() { + e.prepareNext(objectClose) + e.out = append(e.out, '}') +} + +// WriteName writes out the given string in JSON string value and the name +// separator ':'. Returns error if input string contains invalid UTF-8, which +// should not be likely as protobuf field names should be valid. +func (e *Encoder) WriteName(s string) error { + e.prepareNext(name) + var err error + // Append to output regardless of error. + e.out, err = appendString(e.out, s) + e.out = append(e.out, ':') + return err +} + +// StartArray writes out the '[' symbol. +func (e *Encoder) StartArray() { + e.prepareNext(arrayOpen) + e.out = append(e.out, '[') +} + +// EndArray writes out the ']' symbol. +func (e *Encoder) EndArray() { + e.prepareNext(arrayClose) + e.out = append(e.out, ']') +} + +// prepareNext adds possible comma and indentation for the next value based +// on last type and indent option. It also updates lastKind to next. +func (e *Encoder) prepareNext(next kind) { + defer func() { + // Set lastKind to next. + e.lastKind = next + }() + + if len(e.indent) == 0 { + // Need to add comma on the following condition. + if e.lastKind&(scalar|objectClose|arrayClose) != 0 && + next&(name|scalar|objectOpen|arrayOpen) != 0 { + e.out = append(e.out, ',') + } + return + } + + switch { + case e.lastKind&(objectOpen|arrayOpen) != 0: + // If next type is NOT closing, add indent and newline. + if next&(objectClose|arrayClose) == 0 { + e.indents = append(e.indents, e.indent...) + e.out = append(e.out, '\n') + e.out = append(e.out, e.indents...) + } + + case e.lastKind&(scalar|objectClose|arrayClose) != 0: + switch { + // If next type is either a value or name, add comma and newline. + case next&(name|scalar|objectOpen|arrayOpen) != 0: + e.out = append(e.out, ',', '\n') + + // If next type is a closing object or array, adjust indentation. + case next&(objectClose|arrayClose) != 0: + e.indents = e.indents[:len(e.indents)-len(e.indent)] + e.out = append(e.out, '\n') + } + e.out = append(e.out, e.indents...) + + case e.lastKind&name != 0: + e.out = append(e.out, ' ') + } +} diff --git a/vendor/go.temporal.io/api/internal/protojson/json_marshal.go b/vendor/go.temporal.io/api/internal/protojson/json_marshal.go new file mode 100644 index 00000000000..cd2c45b1dd6 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/json_marshal.go @@ -0,0 +1,358 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package protojson + +import ( + "encoding/base64" + "fmt" + + "go.temporal.io/api/internal/protojson/errors" + "go.temporal.io/api/internal/protojson/genid" + "go.temporal.io/api/internal/protojson/json" + "go.temporal.io/api/internal/protojson/order" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/reflect/protoreflect" + "google.golang.org/protobuf/reflect/protoregistry" +) + +const defaultIndent = " " + +// MarshalOptions is a configurable JSON format marshaler. +type MarshalOptions struct { + // Metadata is used for storing request metadata, such as whether shorthand + // payloads are disabled + Metadata map[string]interface{} + + // Multiline specifies whether the marshaler should format the output in + // indented-form with every textual element on a new line. + // If Indent is an empty string, then an arbitrary indent is chosen. + Multiline bool + + // Indent specifies the set of indentation characters to use in a multiline + // formatted output such that every entry is preceded by Indent and + // terminated by a newline. If non-empty, then Multiline is treated as true. + // Indent can only be composed of space or tab characters. + Indent string + + // AllowPartial allows messages that have missing required fields to marshal + // without returning an error. If AllowPartial is false (the default), + // Marshal will return error if there are any missing required fields. + AllowPartial bool + + // UseProtoNames uses proto field name instead of lowerCamelCase name in JSON + // field names. + UseProtoNames bool + + // UseEnumNumbers emits enum values as numbers. + UseEnumNumbers bool + + // EmitUnpopulated specifies whether to emit unpopulated fields. It does not + // emit unpopulated oneof fields or unpopulated extension fields. + // The JSON value emitted for unpopulated fields are as follows: + // ╔═══════╤════════════════════════════╗ + // ║ JSON │ Protobuf field ║ + // ╠═══════╪════════════════════════════╣ + // ║ false │ proto3 boolean fields ║ + // ║ 0 │ proto3 numeric fields ║ + // ║ "" │ proto3 string/bytes fields ║ + // ║ null │ proto2 scalar fields ║ + // ║ null │ message fields ║ + // ║ [] │ list fields ║ + // ║ {} │ map fields ║ + // ╚═══════╧════════════════════════════╝ + EmitUnpopulated bool + + // EmitDefaultValues specifies whether to emit default-valued primitive fields, + // empty lists, and empty maps. The fields affected are as follows: + // ╔═══════╤════════════════════════════════════════╗ + // ║ JSON │ Protobuf field ║ + // ╠═══════╪════════════════════════════════════════╣ + // ║ false │ non-optional scalar boolean fields ║ + // ║ 0 │ non-optional scalar numeric fields ║ + // ║ "" │ non-optional scalar string/byte fields ║ + // ║ [] │ empty repeated fields ║ + // ║ {} │ empty map fields ║ + // ╚═══════╧════════════════════════════════════════╝ + // + // Behaves similarly to EmitUnpopulated, but does not emit "null"-value fields, + // i.e. presence-sensing fields that are omitted will remain omitted to preserve + // presence-sensing. + // EmitUnpopulated takes precedence over EmitDefaultValues since the former generates + // a strict superset of the latter. + EmitDefaultValues bool + + // Resolver is used for looking up types when expanding google.protobuf.Any + // messages. If nil, this defaults to using protoregistry.GlobalTypes. + Resolver interface { + protoregistry.ExtensionTypeResolver + protoregistry.MessageTypeResolver + } +} + +// Marshal marshals the given [proto.Message] in the JSON format using options in +// MarshalOptions. Do not depend on the output being stable. It may change over +// time across different versions of the program. +func (o MarshalOptions) Marshal(m proto.Message) ([]byte, error) { + return o.marshal(nil, m) +} + +// MarshalAppend appends the JSON format encoding of m to b, +// returning the result. +func (o MarshalOptions) MarshalAppend(b []byte, m proto.Message) ([]byte, error) { + return o.marshal(b, m) +} + +// marshal is a centralized function that all marshal operations go through. +// For profiling purposes, avoid changing the name of this function or +// introducing other code paths for marshal that do not go through this. +func (o MarshalOptions) marshal(b []byte, m proto.Message) ([]byte, error) { + if o.Multiline && o.Indent == "" { + o.Indent = defaultIndent + } + if o.Resolver == nil { + o.Resolver = protoregistry.GlobalTypes + } + + // TODO: we could presize the buffer here with a best-guess to reduce allocations + internalEnc, err := json.NewEncoder(b, o.Indent) + if err != nil { + return nil, err + } + + // Treat nil message interface as an empty message, + // in which case the output in an empty JSON object. + if m == nil { + return append(b, '{', '}'), nil + } + + enc := encoder{internalEnc, o} + if err := enc.marshalMessage(m.ProtoReflect(), ""); err != nil { + return nil, err + } + if o.AllowPartial { + return enc.Bytes(), nil + } + return enc.Bytes(), proto.CheckInitialized(m) +} + +type encoder struct { + *json.Encoder + opts MarshalOptions +} + +// unpopulatedFieldRanger wraps a protoreflect.Message and modifies its Range +// method to additionally iterate over unpopulated fields. +type unpopulatedFieldRanger struct { + protoreflect.Message + + skipNull bool +} + +func (m unpopulatedFieldRanger) Range(f func(protoreflect.FieldDescriptor, protoreflect.Value) bool) { + fds := m.Descriptor().Fields() + for i := 0; i < fds.Len(); i++ { + fd := fds.Get(i) + if m.Has(fd) || fd.ContainingOneof() != nil { + continue // ignore populated fields and fields within a oneofs + } + + v := m.Get(fd) + isProto2Scalar := fd.Syntax() == protoreflect.Proto2 && fd.Default().IsValid() + isSingularMessage := fd.Cardinality() != protoreflect.Repeated && fd.Message() != nil + if isProto2Scalar || isSingularMessage { + if m.skipNull { + continue + } + v = protoreflect.Value{} // use invalid value to emit null + } + if !f(fd, v) { + return + } + } + m.Message.Range(f) +} + +// marshalMessage marshals the fields in the given protoreflect.Message. +// If the typeURL is non-empty, then a synthetic "@type" field is injected +// containing the URL as the value. +func (e encoder) marshalMessage(m protoreflect.Message, typeURL string) error { + if marshal := wellKnownTypeMarshaler(m.Descriptor().FullName()); marshal != nil { + return marshal(e, m) + } + if jsu, ok := m.Interface().(ProtoJSONMaybeMarshaler); ok { + if handled, err := jsu.MaybeMarshalProtoJSON(e.opts.Metadata, e.Encoder); handled { + if err != nil { + return fmt.Errorf("MaybeMarshalProtoJSON impl for %T failed: %w", jsu, err) + } + return nil + } + } + + e.StartObject() + defer e.EndObject() + + var fields order.FieldRanger = m + switch { + case e.opts.EmitUnpopulated: + fields = unpopulatedFieldRanger{Message: m, skipNull: false} + case e.opts.EmitDefaultValues: + fields = unpopulatedFieldRanger{Message: m, skipNull: true} + } + + var err error + if typeURL != "" { + // We manually marshal the synthetic @type field here as we cannot create a synthetic FieldDescriptor. + // We cannot implement the ProtoInternal method as it takes an internal-only type as a parameter... + if err := e.WriteName("@type"); err != nil { + return err + } + if e.WriteString(typeURL) != nil { + return errors.InvalidUTF8("@type") + } + } + order.RangeFields(fields, order.IndexNameFieldOrder, func(fd protoreflect.FieldDescriptor, v protoreflect.Value) bool { + name := fd.JSONName() + if e.opts.UseProtoNames { + name = fd.TextName() + } + + if err = e.WriteName(name); err != nil { + return false + } + if err = e.marshalValue(v, fd); err != nil { + return false + } + return true + }) + return err +} + +// marshalValue marshals the given protoreflect.Value. +func (e encoder) marshalValue(val protoreflect.Value, fd protoreflect.FieldDescriptor) error { + switch { + case fd.IsList(): + return e.marshalList(val.List(), fd) + case fd.IsMap(): + return e.marshalMap(val.Map(), fd) + default: + return e.marshalSingular(val, fd) + } +} + +// marshalSingular marshals the given non-repeated field value. This includes +// all scalar types, enums, messages, and groups. +func (e encoder) marshalSingular(val protoreflect.Value, fd protoreflect.FieldDescriptor) error { + if !val.IsValid() { + e.WriteNull() + return nil + } + + switch kind := fd.Kind(); kind { + case protoreflect.BoolKind: + e.WriteBool(val.Bool()) + + case protoreflect.StringKind: + if e.WriteString(val.String()) != nil { + return errors.InvalidUTF8(string(fd.FullName())) + } + + case protoreflect.Int32Kind, protoreflect.Sint32Kind, protoreflect.Sfixed32Kind: + e.WriteInt(val.Int()) + + case protoreflect.Uint32Kind, protoreflect.Fixed32Kind: + e.WriteUint(val.Uint()) + + case protoreflect.Int64Kind, protoreflect.Sint64Kind, protoreflect.Uint64Kind, + protoreflect.Sfixed64Kind, protoreflect.Fixed64Kind: + // 64-bit integers are written out as JSON string. + e.WriteString(val.String()) + + case protoreflect.FloatKind: + // Encoder.WriteFloat handles the special numbers NaN and infinites. + e.WriteFloat(val.Float(), 32) + + case protoreflect.DoubleKind: + // Encoder.WriteFloat handles the special numbers NaN and infinites. + e.WriteFloat(val.Float(), 64) + + case protoreflect.BytesKind: + e.WriteString(base64.StdEncoding.EncodeToString(val.Bytes())) + + case protoreflect.EnumKind: + if fd.Enum().FullName() == genid.NullValue_enum_fullname { + e.WriteNull() + } else { + desc := fd.Enum().Values().ByNumber(val.Enum()) + if e.opts.UseEnumNumbers || desc == nil { + e.WriteInt(int64(val.Enum())) + } else { + e.WriteString(string(desc.Name())) + } + } + + case protoreflect.MessageKind, protoreflect.GroupKind: + if err := e.marshalMessage(val.Message(), ""); err != nil { + return err + } + + default: + panic(fmt.Sprintf("%v has unknown kind: %v", fd.FullName(), kind)) + } + return nil +} + +// marshalList marshals the given protoreflect.List. +func (e encoder) marshalList(list protoreflect.List, fd protoreflect.FieldDescriptor) error { + e.StartArray() + defer e.EndArray() + + for i := 0; i < list.Len(); i++ { + item := list.Get(i) + if err := e.marshalSingular(item, fd); err != nil { + return err + } + } + return nil +} + +// marshalMap marshals given protoreflect.Map. +func (e encoder) marshalMap(mmap protoreflect.Map, fd protoreflect.FieldDescriptor) error { + e.StartObject() + defer e.EndObject() + + var err error + order.RangeEntries(mmap, order.GenericKeyOrder, func(k protoreflect.MapKey, v protoreflect.Value) bool { + if err = e.WriteName(k.String()); err != nil { + return false + } + if err = e.marshalSingular(v, fd.MapValue()); err != nil { + return false + } + return true + }) + return err +} diff --git a/vendor/go.temporal.io/api/internal/protojson/json_unmarshal.go b/vendor/go.temporal.io/api/internal/protojson/json_unmarshal.go new file mode 100644 index 00000000000..c7d7d10b16a --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/json_unmarshal.go @@ -0,0 +1,723 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package protojson + +import ( + "encoding/base64" + "fmt" + "math" + "runtime" + "strconv" + "strings" + + "google.golang.org/protobuf/encoding/protowire" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/reflect/protoreflect" + "google.golang.org/protobuf/reflect/protoregistry" + + "go.temporal.io/api/internal/protojson/errors" + "go.temporal.io/api/internal/protojson/genid" + "go.temporal.io/api/internal/protojson/json" + "go.temporal.io/api/internal/protojson/set" + "go.temporal.io/api/internal/strcase" +) + +// Unmarshal reads the given []byte into the given [proto.Message]. +// The provided message must be mutable (e.g., a non-nil pointer to a message). +func Unmarshal(b []byte, m proto.Message) error { + return UnmarshalOptions{}.Unmarshal(b, m) +} + +// UnmarshalOptions is a configurable JSON format parser. +type UnmarshalOptions struct { + // Metadata is used for storing request metadata, such as whether shorthand + // payloads are disabled + Metadata map[string]interface{} + + // If AllowPartial is set, input for messages that will result in missing + // required fields will not return an error. + AllowPartial bool + + // If DiscardUnknown is set, unknown fields and enum name values are ignored. + DiscardUnknown bool + + // Resolver is used for looking up types when unmarshaling + // google.protobuf.Any messages or extension fields. + // If nil, this defaults to using protoregistry.GlobalTypes. + Resolver interface { + protoregistry.MessageTypeResolver + protoregistry.ExtensionTypeResolver + } + + // RecursionLimit limits how deeply messages may be nested. + // If zero, a default limit is applied. + RecursionLimit int +} + +// Unmarshal reads the given []byte and populates the given [proto.Message] +// using options in the UnmarshalOptions object. +// It will clear the message first before setting the fields. +// If it returns an error, the given message may be partially set. +// The provided message must be mutable (e.g., a non-nil pointer to a message). +func (o UnmarshalOptions) Unmarshal(b []byte, m proto.Message) error { + return o.unmarshal(b, m) +} + +// unmarshal is a centralized function that all unmarshal operations go through. +// For profiling purposes, avoid changing the name of this function or +// introducing other code paths for unmarshal that do not go through this. +func (o UnmarshalOptions) unmarshal(b []byte, m proto.Message) error { + proto.Reset(m) + + if o.Resolver == nil { + o.Resolver = protoregistry.GlobalTypes + } + + if o.RecursionLimit == 0 { + o.RecursionLimit = protowire.DefaultRecursionLimit + } + + dec := decoder{json.NewDecoder(b), o} + if err := dec.unmarshalMessage(m.ProtoReflect(), false); err != nil { + return err + } + + // Check for EOF. + tok, err := dec.Read() + if err != nil { + return err + } + if tok.Kind() != json.EOF { + return dec.unexpectedTokenError(tok) + } + + if o.AllowPartial { + return nil + } + return proto.CheckInitialized(m) +} + +type decoder struct { + *json.Decoder + opts UnmarshalOptions +} + +// newError returns an error object with position info. +func (d decoder) newError(pos int, f string, x ...interface{}) error { + line, column := d.Position(pos) + head := fmt.Sprintf("(line %d:%d): ", line, column) + return errors.New(head+f, x...) +} + +// unexpectedTokenError returns a syntax error for the given unexpected token. +func (d decoder) unexpectedTokenError(tok json.Token) error { + pc, _, _, ok := runtime.Caller(1) + var fl string + var line int + if ok { + f := runtime.FuncForPC(pc) + fl, line = f.FileLine(pc) + } + return d.syntaxError(tok.Pos(), "unexpected token %s (in %s:%d)", tok.RawString(), fl, line) +} + +// syntaxError returns a syntax error for given position. +func (d decoder) syntaxError(pos int, f string, x ...interface{}) error { + line, column := d.Position(pos) + head := fmt.Sprintf("syntax error (line %d:%d): ", line, column) + return errors.New(head+f, x...) +} + +// unmarshalMessage unmarshals a message into the given protoreflect.Message. +func (d decoder) unmarshalMessage(m protoreflect.Message, skipTypeURL bool) error { + d.opts.RecursionLimit-- + if d.opts.RecursionLimit < 0 { + return errors.New("exceeded max recursion depth") + } + + if jsu, ok := m.Interface().(ProtoJSONMaybeUnmarshaler); ok { + if handled, err := jsu.MaybeUnmarshalProtoJSON(d.opts.Metadata, d.Decoder); handled || err != nil { + return err + } + } + if unmarshal := wellKnownTypeUnmarshaler(m.Descriptor().FullName()); unmarshal != nil { + return unmarshal(d, m) + } + + tok, err := d.Read() + if err != nil { + return err + } + if tok.Kind() != json.ObjectOpen { + return d.unexpectedTokenError(tok) + } + + messageDesc := m.Descriptor() + var seenNums set.Ints + var seenOneofs set.Ints + fieldDescs := messageDesc.Fields() + for { + // Read field name. + tok, err := d.Read() + if err != nil { + return err + } + switch tok.Kind() { + default: + return d.unexpectedTokenError(tok) + case json.ObjectClose: + return nil + case json.Name: + // Continue below. + } + + name := tok.Name() + // Unmarshaling a non-custom embedded message in Any will contain the + // JSON field "@type" which should be skipped because it is not a field + // of the embedded message, but simply an artifact of the Any format. + if skipTypeURL && name == "@type" { + d.Read() + continue + } + + // Get the FieldDescriptor. + var fd protoreflect.FieldDescriptor + if strings.HasPrefix(name, "[") && strings.HasSuffix(name, "]") { + // Only extension names are in [name] format. + extName := protoreflect.FullName(name[1 : len(name)-1]) + extType, err := d.opts.Resolver.FindExtensionByName(extName) + if err != nil && err != protoregistry.NotFound { + return d.newError(tok.Pos(), "unable to resolve %s: %v", tok.RawString(), err) + } + if extType != nil { + fd = extType.TypeDescriptor() + if !messageDesc.ExtensionRanges().Has(fd.Number()) || fd.ContainingMessage().FullName() != messageDesc.FullName() { + return d.newError(tok.Pos(), "message %v cannot be extended by %v", messageDesc.FullName(), fd.FullName()) + } + } + } else { + // The name can either be the JSON name or the proto field name. + fd = fieldDescs.ByJSONName(name) + if fd == nil { + fd = fieldDescs.ByTextName(name) + } + } + + if fd == nil { + // Field is unknown. + if d.opts.DiscardUnknown { + if err := d.skipJSONValue(); err != nil { + return err + } + continue + } + return d.newError(tok.Pos(), "unknown field %v", tok.RawString()) + } + + // Do not allow duplicate fields. + num := uint64(fd.Number()) + if seenNums.Has(num) { + return d.newError(tok.Pos(), "duplicate field %v", tok.RawString()) + } + seenNums.Set(num) + + // No need to set values for JSON null unless the field type is + // google.protobuf.Value or google.protobuf.NullValue. + if tok, _ := d.Peek(); tok.Kind() == json.Null && !isKnownValue(fd) && !isNullValue(fd) { + d.Read() + continue + } + + switch { + case fd.IsList(): + list := m.Mutable(fd).List() + if err := d.unmarshalList(list, fd); err != nil { + return err + } + case fd.IsMap(): + mmap := m.Mutable(fd).Map() + if err := d.unmarshalMap(mmap, fd); err != nil { + return err + } + default: + // If field is a oneof, check if it has already been set. + if od := fd.ContainingOneof(); od != nil { + idx := uint64(od.Index()) + if seenOneofs.Has(idx) { + return d.newError(tok.Pos(), "error parsing %s, oneof %v is already set", tok.RawString(), od.FullName()) + } + seenOneofs.Set(idx) + } + + // Required or optional fields. + if err := d.unmarshalSingular(m, fd); err != nil { + return err + } + } + } +} + +func isKnownValue(fd protoreflect.FieldDescriptor) bool { + md := fd.Message() + return md != nil && md.FullName() == genid.Value_message_fullname +} + +func isNullValue(fd protoreflect.FieldDescriptor) bool { + ed := fd.Enum() + return ed != nil && ed.FullName() == genid.NullValue_enum_fullname +} + +// unmarshalSingular unmarshals to the non-repeated field specified +// by the given FieldDescriptor. +func (d decoder) unmarshalSingular(m protoreflect.Message, fd protoreflect.FieldDescriptor) error { + var val protoreflect.Value + var err error + switch fd.Kind() { + case protoreflect.MessageKind, protoreflect.GroupKind: + val = m.NewField(fd) + err = d.unmarshalMessage(val.Message(), false) + default: + val, err = d.unmarshalScalar(fd) + } + + if err != nil { + return err + } + if val.IsValid() { + m.Set(fd, val) + } + return nil +} + +// unmarshalScalar unmarshals to a scalar/enum protoreflect.Value specified by +// the given FieldDescriptor. +func (d decoder) unmarshalScalar(fd protoreflect.FieldDescriptor) (protoreflect.Value, error) { + const b32 int = 32 + const b64 int = 64 + + tok, err := d.Read() + if err != nil { + return protoreflect.Value{}, err + } + + kind := fd.Kind() + switch kind { + case protoreflect.BoolKind: + if tok.Kind() == json.Bool { + return protoreflect.ValueOfBool(tok.Bool()), nil + } + + case protoreflect.Int32Kind, protoreflect.Sint32Kind, protoreflect.Sfixed32Kind: + if v, ok := unmarshalInt(tok, b32); ok { + return v, nil + } + + case protoreflect.Int64Kind, protoreflect.Sint64Kind, protoreflect.Sfixed64Kind: + if v, ok := unmarshalInt(tok, b64); ok { + return v, nil + } + + case protoreflect.Uint32Kind, protoreflect.Fixed32Kind: + if v, ok := unmarshalUint(tok, b32); ok { + return v, nil + } + + case protoreflect.Uint64Kind, protoreflect.Fixed64Kind: + if v, ok := unmarshalUint(tok, b64); ok { + return v, nil + } + + case protoreflect.FloatKind: + if v, ok := unmarshalFloat(tok, b32); ok { + return v, nil + } + + case protoreflect.DoubleKind: + if v, ok := unmarshalFloat(tok, b64); ok { + return v, nil + } + + case protoreflect.StringKind: + if tok.Kind() == json.String { + return protoreflect.ValueOfString(tok.ParsedString()), nil + } + + case protoreflect.BytesKind: + if v, ok := unmarshalBytes(tok); ok { + return v, nil + } + + case protoreflect.EnumKind: + if v, ok := unmarshalEnum(tok, fd, d.opts.DiscardUnknown); ok { + return v, nil + } + + default: + panic(fmt.Sprintf("unmarshalScalar: invalid scalar kind %v", kind)) + } + + return protoreflect.Value{}, d.newError(tok.Pos(), "invalid value for %v type: %v", kind, tok.RawString()) +} + +func unmarshalInt(tok json.Token, bitSize int) (protoreflect.Value, bool) { + switch tok.Kind() { + case json.Number: + return getInt(tok, bitSize) + + case json.String: + // Decode number from string. + s := strings.TrimSpace(tok.ParsedString()) + if len(s) != len(tok.ParsedString()) { + return protoreflect.Value{}, false + } + dec := json.NewDecoder([]byte(s)) + tok, err := dec.Read() + if err != nil { + return protoreflect.Value{}, false + } + return getInt(tok, bitSize) + } + return protoreflect.Value{}, false +} + +func getInt(tok json.Token, bitSize int) (protoreflect.Value, bool) { + n, ok := tok.Int(bitSize) + if !ok { + return protoreflect.Value{}, false + } + if bitSize == 32 { + return protoreflect.ValueOfInt32(int32(n)), true + } + return protoreflect.ValueOfInt64(n), true +} + +func unmarshalUint(tok json.Token, bitSize int) (protoreflect.Value, bool) { + switch tok.Kind() { + case json.Number: + return getUint(tok, bitSize) + + case json.String: + // Decode number from string. + s := strings.TrimSpace(tok.ParsedString()) + if len(s) != len(tok.ParsedString()) { + return protoreflect.Value{}, false + } + dec := json.NewDecoder([]byte(s)) + tok, err := dec.Read() + if err != nil { + return protoreflect.Value{}, false + } + return getUint(tok, bitSize) + } + return protoreflect.Value{}, false +} + +func getUint(tok json.Token, bitSize int) (protoreflect.Value, bool) { + n, ok := tok.Uint(bitSize) + if !ok { + return protoreflect.Value{}, false + } + if bitSize == 32 { + return protoreflect.ValueOfUint32(uint32(n)), true + } + return protoreflect.ValueOfUint64(n), true +} + +func unmarshalFloat(tok json.Token, bitSize int) (protoreflect.Value, bool) { + switch tok.Kind() { + case json.Number: + return getFloat(tok, bitSize) + + case json.String: + s := tok.ParsedString() + switch s { + case "NaN": + if bitSize == 32 { + return protoreflect.ValueOfFloat32(float32(math.NaN())), true + } + return protoreflect.ValueOfFloat64(math.NaN()), true + case "Infinity": + if bitSize == 32 { + return protoreflect.ValueOfFloat32(float32(math.Inf(+1))), true + } + return protoreflect.ValueOfFloat64(math.Inf(+1)), true + case "-Infinity": + if bitSize == 32 { + return protoreflect.ValueOfFloat32(float32(math.Inf(-1))), true + } + return protoreflect.ValueOfFloat64(math.Inf(-1)), true + } + + // Decode number from string. + if len(s) != len(strings.TrimSpace(s)) { + return protoreflect.Value{}, false + } + dec := json.NewDecoder([]byte(s)) + tok, err := dec.Read() + if err != nil { + return protoreflect.Value{}, false + } + return getFloat(tok, bitSize) + } + return protoreflect.Value{}, false +} + +func getFloat(tok json.Token, bitSize int) (protoreflect.Value, bool) { + n, ok := tok.Float(bitSize) + if !ok { + return protoreflect.Value{}, false + } + if bitSize == 32 { + return protoreflect.ValueOfFloat32(float32(n)), true + } + return protoreflect.ValueOfFloat64(n), true +} + +func unmarshalBytes(tok json.Token) (protoreflect.Value, bool) { + if tok.Kind() != json.String { + return protoreflect.Value{}, false + } + + s := tok.ParsedString() + enc := base64.StdEncoding + if strings.ContainsAny(s, "-_") { + enc = base64.URLEncoding + } + if len(s)%4 != 0 { + enc = enc.WithPadding(base64.NoPadding) + } + b, err := enc.DecodeString(s) + if err != nil { + return protoreflect.Value{}, false + } + return protoreflect.ValueOfBytes(b), true +} + +func unmarshalEnum(tok json.Token, fd protoreflect.FieldDescriptor, discardUnknown bool) (protoreflect.Value, bool) { + switch tok.Kind() { + case json.String: + // Lookup EnumNumber based on name. + // We need to support both the canonical JSON format (SCREAMING_SNAKE_ENUMS) as well + // as our older CamelCaseEnums form for compatibility, so we try both variants + s := tok.ParsedString() + if enumVal := fd.Enum().Values().ByName(protoreflect.Name(s)); enumVal != nil { + return protoreflect.ValueOfEnum(enumVal.Number()), true + } + + // If not found we'll assume it's old-style and prepend TYPE_PREFIX_ + s = strcase.ToScreamingSnake(fmt.Sprintf("%s_%s", string(fd.Enum().Name()), s)) + if enumVal := fd.Enum().Values().ByName(protoreflect.Name(s)); enumVal != nil { + return protoreflect.ValueOfEnum(enumVal.Number()), true + } + + if discardUnknown { + return protoreflect.Value{}, true + } + + case json.Number: + if n, ok := tok.Int(32); ok { + return protoreflect.ValueOfEnum(protoreflect.EnumNumber(n)), true + } + + case json.Null: + // This is only valid for google.protobuf.NullValue. + if isNullValue(fd) { + return protoreflect.ValueOfEnum(0), true + } + } + + return protoreflect.Value{}, false +} + +func (d decoder) unmarshalList(list protoreflect.List, fd protoreflect.FieldDescriptor) error { + tok, err := d.Read() + if err != nil { + return err + } + if tok.Kind() != json.ArrayOpen { + return d.unexpectedTokenError(tok) + } + + switch fd.Kind() { + case protoreflect.MessageKind, protoreflect.GroupKind: + for { + tok, err := d.Peek() + if err != nil { + return err + } + + if tok.Kind() == json.ArrayClose { + d.Read() + return nil + } + + val := list.NewElement() + if err := d.unmarshalMessage(val.Message(), false); err != nil { + return err + } + list.Append(val) + } + default: + for { + tok, err := d.Peek() + if err != nil { + return err + } + + if tok.Kind() == json.ArrayClose { + d.Read() + return nil + } + + val, err := d.unmarshalScalar(fd) + if err != nil { + return err + } + if val.IsValid() { + list.Append(val) + } + } + } + + return nil +} + +func (d decoder) unmarshalMap(mmap protoreflect.Map, fd protoreflect.FieldDescriptor) error { + tok, err := d.Read() + if err != nil { + return err + } + if tok.Kind() != json.ObjectOpen { + return d.unexpectedTokenError(tok) + } + + // Determine ahead whether map entry is a scalar type or a message type in + // order to call the appropriate unmarshalMapValue func inside the for loop + // below. + var unmarshalMapValue func() (protoreflect.Value, error) + switch fd.MapValue().Kind() { + case protoreflect.MessageKind, protoreflect.GroupKind: + unmarshalMapValue = func() (protoreflect.Value, error) { + val := mmap.NewValue() + if err := d.unmarshalMessage(val.Message(), false); err != nil { + return protoreflect.Value{}, err + } + return val, nil + } + default: + unmarshalMapValue = func() (protoreflect.Value, error) { + return d.unmarshalScalar(fd.MapValue()) + } + } + +Loop: + for { + // Read field name. + tok, err := d.Read() + if err != nil { + return err + } + switch tok.Kind() { + default: + return d.unexpectedTokenError(tok) + case json.ObjectClose: + break Loop + case json.Name: + // Continue. + } + + // Unmarshal field name. + pkey, err := d.unmarshalMapKey(tok, fd.MapKey()) + if err != nil { + return err + } + + // Check for duplicate field name. + if mmap.Has(pkey) { + return d.newError(tok.Pos(), "duplicate map key %v", tok.RawString()) + } + + // Read and unmarshal field value. + pval, err := unmarshalMapValue() + if err != nil { + return err + } + if pval.IsValid() { + mmap.Set(pkey, pval) + } + } + + return nil +} + +// unmarshalMapKey converts given token of Name kind into a protoreflect.MapKey. +// A map key type is any integral or string type. +func (d decoder) unmarshalMapKey(tok json.Token, fd protoreflect.FieldDescriptor) (protoreflect.MapKey, error) { + const b32 = 32 + const b64 = 64 + const base10 = 10 + + name := tok.Name() + kind := fd.Kind() + switch kind { + case protoreflect.StringKind: + return protoreflect.ValueOfString(name).MapKey(), nil + + case protoreflect.BoolKind: + switch name { + case "true": + return protoreflect.ValueOfBool(true).MapKey(), nil + case "false": + return protoreflect.ValueOfBool(false).MapKey(), nil + } + + case protoreflect.Int32Kind, protoreflect.Sint32Kind, protoreflect.Sfixed32Kind: + if n, err := strconv.ParseInt(name, base10, b32); err == nil { + return protoreflect.ValueOfInt32(int32(n)).MapKey(), nil + } + + case protoreflect.Int64Kind, protoreflect.Sint64Kind, protoreflect.Sfixed64Kind: + if n, err := strconv.ParseInt(name, base10, b64); err == nil { + return protoreflect.ValueOfInt64(int64(n)).MapKey(), nil + } + + case protoreflect.Uint32Kind, protoreflect.Fixed32Kind: + if n, err := strconv.ParseUint(name, base10, b32); err == nil { + return protoreflect.ValueOfUint32(uint32(n)).MapKey(), nil + } + + case protoreflect.Uint64Kind, protoreflect.Fixed64Kind: + if n, err := strconv.ParseUint(name, base10, b64); err == nil { + return protoreflect.ValueOfUint64(uint64(n)).MapKey(), nil + } + + default: + panic(fmt.Sprintf("invalid kind for map key: %v", kind)) + } + + return protoreflect.MapKey{}, d.newError(tok.Pos(), "invalid value for %v key: %s", kind, tok.RawString()) +} diff --git a/vendor/go.temporal.io/api/internal/protojson/maybe_marshal.go b/vendor/go.temporal.io/api/internal/protojson/maybe_marshal.go new file mode 100644 index 00000000000..05cbffe0adc --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/maybe_marshal.go @@ -0,0 +1,43 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package protojson + +import "go.temporal.io/api/internal/protojson/json" + +// ProtoJSONMaybeMarshaler is implemented by any proto struct that wants to +// customize optional Temporal-specific JSON conversion. +type ProtoJSONMaybeMarshaler interface { + // MaybeMarshalProtoJSON is for formatting the proto message as JSON. If the + // "handled" result value is false, "err" are ignored and the default + // protojson behavior occurs. + MaybeMarshalProtoJSON(meta map[string]interface{}, enc *json.Encoder) (handled bool, err error) +} + +// ProtoJSONMaybeUnmarshaler is implemented by any proto struct that wants to +// customize optional Temporal-specific JSON conversion. +type ProtoJSONMaybeUnmarshaler interface { + // MaybeUnmarshalProtoJSON is for parsing the given JSON into the proto message. + // If the "handled" result value is false, "err" is ignored and the default + // protojson unmarshaling proceeds + MaybeUnmarshalProtoJSON(meta map[string]interface{}, dec *json.Decoder) (handled bool, err error) +} diff --git a/vendor/go.temporal.io/api/internal/protojson/order/order.go b/vendor/go.temporal.io/api/internal/protojson/order/order.go new file mode 100644 index 00000000000..dea522e127d --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/order/order.go @@ -0,0 +1,89 @@ +// Copyright 2020 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package order + +import ( + "google.golang.org/protobuf/reflect/protoreflect" +) + +// FieldOrder specifies the ordering to visit message fields. +// It is a function that reports whether x is ordered before y. +type FieldOrder func(x, y protoreflect.FieldDescriptor) bool + +var ( + // AnyFieldOrder specifies no specific field ordering. + AnyFieldOrder FieldOrder = nil + + // LegacyFieldOrder sorts fields in the same ordering as emitted by + // wire serialization in the github.com/golang/protobuf implementation. + LegacyFieldOrder FieldOrder = func(x, y protoreflect.FieldDescriptor) bool { + ox, oy := x.ContainingOneof(), y.ContainingOneof() + inOneof := func(od protoreflect.OneofDescriptor) bool { + return od != nil && !od.IsSynthetic() + } + + // Extension fields sort before non-extension fields. + if x.IsExtension() != y.IsExtension() { + return x.IsExtension() && !y.IsExtension() + } + // Fields not within a oneof sort before those within a oneof. + if inOneof(ox) != inOneof(oy) { + return !inOneof(ox) && inOneof(oy) + } + // Fields in disjoint oneof sets are sorted by declaration index. + if inOneof(ox) && inOneof(oy) && ox != oy { + return ox.Index() < oy.Index() + } + // Fields sorted by field number. + return x.Number() < y.Number() + } + + // NumberFieldOrder sorts fields by their field number. + NumberFieldOrder FieldOrder = func(x, y protoreflect.FieldDescriptor) bool { + return x.Number() < y.Number() + } + + // IndexNameFieldOrder sorts non-extension fields before extension fields. + // Non-extensions are sorted according to their declaration index. + // Extensions are sorted according to their full name. + IndexNameFieldOrder FieldOrder = func(x, y protoreflect.FieldDescriptor) bool { + // Non-extension fields sort before extension fields. + if x.IsExtension() != y.IsExtension() { + return !x.IsExtension() && y.IsExtension() + } + // Extensions sorted by fullname. + if x.IsExtension() && y.IsExtension() { + return x.FullName() < y.FullName() + } + // Non-extensions sorted by declaration index. + return x.Index() < y.Index() + } +) + +// KeyOrder specifies the ordering to visit map entries. +// It is a function that reports whether x is ordered before y. +type KeyOrder func(x, y protoreflect.MapKey) bool + +var ( + // AnyKeyOrder specifies no specific key ordering. + AnyKeyOrder KeyOrder = nil + + // GenericKeyOrder sorts false before true, numeric keys in ascending order, + // and strings in lexicographical ordering according to UTF-8 codepoints. + GenericKeyOrder KeyOrder = func(x, y protoreflect.MapKey) bool { + switch x.Interface().(type) { + case bool: + return !x.Bool() && y.Bool() + case int32, int64: + return x.Int() < y.Int() + case uint32, uint64: + return x.Uint() < y.Uint() + case string: + return x.String() < y.String() + default: + panic("invalid map key type") + } + } +) diff --git a/vendor/go.temporal.io/api/internal/protojson/order/range.go b/vendor/go.temporal.io/api/internal/protojson/order/range.go new file mode 100644 index 00000000000..1665a68e5b7 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/order/range.go @@ -0,0 +1,115 @@ +// Copyright 2020 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package order provides ordered access to messages and maps. +package order + +import ( + "sort" + "sync" + + "google.golang.org/protobuf/reflect/protoreflect" +) + +type messageField struct { + fd protoreflect.FieldDescriptor + v protoreflect.Value +} + +var messageFieldPool = sync.Pool{ + New: func() interface{} { return new([]messageField) }, +} + +type ( + // FieldRnger is an interface for visiting all fields in a message. + // The protoreflect.Message type implements this interface. + FieldRanger interface{ Range(VisitField) } + // VisitField is called every time a message field is visited. + VisitField = func(protoreflect.FieldDescriptor, protoreflect.Value) bool +) + +// RangeFields iterates over the fields of fs according to the specified order. +func RangeFields(fs FieldRanger, less FieldOrder, fn VisitField) { + if less == nil { + fs.Range(fn) + return + } + + // Obtain a pre-allocated scratch buffer. + p := messageFieldPool.Get().(*[]messageField) + fields := (*p)[:0] + defer func() { + if cap(fields) < 1024 { + *p = fields + messageFieldPool.Put(p) + } + }() + + // Collect all fields in the message and sort them. + fs.Range(func(fd protoreflect.FieldDescriptor, v protoreflect.Value) bool { + fields = append(fields, messageField{fd, v}) + return true + }) + sort.Slice(fields, func(i, j int) bool { + return less(fields[i].fd, fields[j].fd) + }) + + // Visit the fields in the specified ordering. + for _, f := range fields { + if !fn(f.fd, f.v) { + return + } + } +} + +type mapEntry struct { + k protoreflect.MapKey + v protoreflect.Value +} + +var mapEntryPool = sync.Pool{ + New: func() interface{} { return new([]mapEntry) }, +} + +type ( + // EntryRanger is an interface for visiting all fields in a message. + // The protoreflect.Map type implements this interface. + EntryRanger interface{ Range(VisitEntry) } + // VisitEntry is called every time a map entry is visited. + VisitEntry = func(protoreflect.MapKey, protoreflect.Value) bool +) + +// RangeEntries iterates over the entries of es according to the specified order. +func RangeEntries(es EntryRanger, less KeyOrder, fn VisitEntry) { + if less == nil { + es.Range(fn) + return + } + + // Obtain a pre-allocated scratch buffer. + p := mapEntryPool.Get().(*[]mapEntry) + entries := (*p)[:0] + defer func() { + if cap(entries) < 1024 { + *p = entries + mapEntryPool.Put(p) + } + }() + + // Collect all entries in the map and sort them. + es.Range(func(k protoreflect.MapKey, v protoreflect.Value) bool { + entries = append(entries, mapEntry{k, v}) + return true + }) + sort.Slice(entries, func(i, j int) bool { + return less(entries[i].k, entries[j].k) + }) + + // Visit the entries in the specified ordering. + for _, e := range entries { + if !fn(e.k, e.v) { + return + } + } +} diff --git a/vendor/go.temporal.io/api/internal/protojson/set/ints.go b/vendor/go.temporal.io/api/internal/protojson/set/ints.go new file mode 100644 index 00000000000..d3d7f89ab55 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/set/ints.go @@ -0,0 +1,58 @@ +// Copyright 2018 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package set provides simple set data structures for uint64s. +package set + +import "math/bits" + +// int64s represents a set of integers within the range of 0..63. +type int64s uint64 + +func (bs *int64s) Len() int { + return bits.OnesCount64(uint64(*bs)) +} +func (bs *int64s) Has(n uint64) bool { + return uint64(*bs)&(uint64(1)< 0 +} +func (bs *int64s) Set(n uint64) { + *(*uint64)(bs) |= uint64(1) << n +} +func (bs *int64s) Clear(n uint64) { + *(*uint64)(bs) &^= uint64(1) << n +} + +// Ints represents a set of integers within the range of 0..math.MaxUint64. +type Ints struct { + lo int64s + hi map[uint64]struct{} +} + +func (bs *Ints) Len() int { + return bs.lo.Len() + len(bs.hi) +} +func (bs *Ints) Has(n uint64) bool { + if n < 64 { + return bs.lo.Has(n) + } + _, ok := bs.hi[n] + return ok +} +func (bs *Ints) Set(n uint64) { + if n < 64 { + bs.lo.Set(n) + return + } + if bs.hi == nil { + bs.hi = make(map[uint64]struct{}) + } + bs.hi[n] = struct{}{} +} +func (bs *Ints) Clear(n uint64) { + if n < 64 { + bs.lo.Clear(n) + return + } + delete(bs.hi, n) +} diff --git a/vendor/go.temporal.io/api/internal/protojson/strs/strings.go b/vendor/go.temporal.io/api/internal/protojson/strs/strings.go new file mode 100644 index 00000000000..bab7b86b390 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/strs/strings.go @@ -0,0 +1,190 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package strs provides string manipulation functionality specific to protobuf. +package strs + +import ( + "go/token" + "strings" + "unicode" + "unicode/utf8" + + "google.golang.org/protobuf/reflect/protoreflect" +) + +// EnforceUTF8 reports whether to enforce strict UTF-8 validation. +func EnforceUTF8(fd protoreflect.FieldDescriptor) bool { + return fd.Syntax() == protoreflect.Proto3 +} + +// GoCamelCase camel-cases a protobuf name for use as a Go identifier. +// +// If there is an interior underscore followed by a lower case letter, +// drop the underscore and convert the letter to upper case. +func GoCamelCase(s string) string { + // Invariant: if the next letter is lower case, it must be converted + // to upper case. + // That is, we process a word at a time, where words are marked by _ or + // upper case letter. Digits are treated as words. + var b []byte + for i := 0; i < len(s); i++ { + c := s[i] + switch { + case c == '.' && i+1 < len(s) && isASCIILower(s[i+1]): + // Skip over '.' in ".{{lowercase}}". + case c == '.': + b = append(b, '_') // convert '.' to '_' + case c == '_' && (i == 0 || s[i-1] == '.'): + // Convert initial '_' to ensure we start with a capital letter. + // Do the same for '_' after '.' to match historic behavior. + b = append(b, 'X') // convert '_' to 'X' + case c == '_' && i+1 < len(s) && isASCIILower(s[i+1]): + // Skip over '_' in "_{{lowercase}}". + case isASCIIDigit(c): + b = append(b, c) + default: + // Assume we have a letter now - if not, it's a bogus identifier. + // The next word is a sequence of characters that must start upper case. + if isASCIILower(c) { + c -= 'a' - 'A' // convert lowercase to uppercase + } + b = append(b, c) + + // Accept lower case sequence that follows. + for ; i+1 < len(s) && isASCIILower(s[i+1]); i++ { + b = append(b, s[i+1]) + } + } + } + return string(b) +} + +// GoSanitized converts a string to a valid Go identifier. +func GoSanitized(s string) string { + // Sanitize the input to the set of valid characters, + // which must be '_' or be in the Unicode L or N categories. + s = strings.Map(func(r rune) rune { + if unicode.IsLetter(r) || unicode.IsDigit(r) { + return r + } + return '_' + }, s) + + // Prepend '_' in the event of a Go keyword conflict or if + // the identifier is invalid (does not start in the Unicode L category). + r, _ := utf8.DecodeRuneInString(s) + if token.Lookup(s).IsKeyword() || !unicode.IsLetter(r) { + return "_" + s + } + return s +} + +// JSONCamelCase converts a snake_case identifier to a camelCase identifier, +// according to the protobuf JSON specification. +func JSONCamelCase(s string) string { + var b []byte + var wasUnderscore bool + for i := 0; i < len(s); i++ { // proto identifiers are always ASCII + c := s[i] + if c != '_' { + if wasUnderscore && isASCIILower(c) { + c -= 'a' - 'A' // convert to uppercase + } + b = append(b, c) + } + wasUnderscore = c == '_' + } + return string(b) +} + +// JSONSnakeCase converts a camelCase identifier to a snake_case identifier, +// according to the protobuf JSON specification. +func JSONSnakeCase(s string) string { + var b []byte + for i := 0; i < len(s); i++ { // proto identifiers are always ASCII + c := s[i] + if isASCIIUpper(c) { + b = append(b, '_') + c += 'a' - 'A' // convert to lowercase + } + b = append(b, c) + } + return string(b) +} + +// MapEntryName derives the name of the map entry message given the field name. +// See protoc v3.8.0: src/google/protobuf/descriptor.cc:254-276,6057 +func MapEntryName(s string) string { + var b []byte + upperNext := true + for _, c := range s { + switch { + case c == '_': + upperNext = true + case upperNext: + b = append(b, byte(unicode.ToUpper(c))) + upperNext = false + default: + b = append(b, byte(c)) + } + } + b = append(b, "Entry"...) + return string(b) +} + +// EnumValueName derives the camel-cased enum value name. +// See protoc v3.8.0: src/google/protobuf/descriptor.cc:297-313 +func EnumValueName(s string) string { + var b []byte + upperNext := true + for _, c := range s { + switch { + case c == '_': + upperNext = true + case upperNext: + b = append(b, byte(unicode.ToUpper(c))) + upperNext = false + default: + b = append(b, byte(unicode.ToLower(c))) + upperNext = false + } + } + return string(b) +} + +// TrimEnumPrefix trims the enum name prefix from an enum value name, +// where the prefix is all lowercase without underscores. +// See protoc v3.8.0: src/google/protobuf/descriptor.cc:330-375 +func TrimEnumPrefix(s, prefix string) string { + s0 := s // original input + for len(s) > 0 && len(prefix) > 0 { + if s[0] == '_' { + s = s[1:] + continue + } + if unicode.ToLower(rune(s[0])) != rune(prefix[0]) { + return s0 // no prefix match + } + s, prefix = s[1:], prefix[1:] + } + if len(prefix) > 0 { + return s0 // no prefix match + } + s = strings.TrimLeft(s, "_") + if len(s) == 0 { + return s0 // avoid returning empty string + } + return s +} + +func isASCIILower(c byte) bool { + return 'a' <= c && c <= 'z' +} +func isASCIIUpper(c byte) bool { + return 'A' <= c && c <= 'Z' +} +func isASCIIDigit(c byte) bool { + return '0' <= c && c <= '9' +} diff --git a/vendor/go.temporal.io/api/internal/protojson/strs/strings_pure.go b/vendor/go.temporal.io/api/internal/protojson/strs/strings_pure.go new file mode 100644 index 00000000000..a1f6f333860 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/strs/strings_pure.go @@ -0,0 +1,28 @@ +// Copyright 2018 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +//go:build purego || appengine +// +build purego appengine + +package strs + +import pref "google.golang.org/protobuf/reflect/protoreflect" + +func UnsafeString(b []byte) string { + return string(b) +} + +func UnsafeBytes(s string) []byte { + return []byte(s) +} + +type Builder struct{} + +func (*Builder) AppendFullName(prefix pref.FullName, name pref.Name) pref.FullName { + return prefix.Append(name) +} + +func (*Builder) MakeString(b []byte) string { + return string(b) +} diff --git a/vendor/go.temporal.io/api/internal/protojson/strs/strings_unsafe.go b/vendor/go.temporal.io/api/internal/protojson/strs/strings_unsafe.go new file mode 100644 index 00000000000..61a84d34185 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/strs/strings_unsafe.go @@ -0,0 +1,95 @@ +// Copyright 2018 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +//go:build !purego && !appengine +// +build !purego,!appengine + +package strs + +import ( + "unsafe" + + "google.golang.org/protobuf/reflect/protoreflect" +) + +type ( + stringHeader struct { + Data unsafe.Pointer + Len int + } + sliceHeader struct { + Data unsafe.Pointer + Len int + Cap int + } +) + +// UnsafeString returns an unsafe string reference of b. +// The caller must treat the input slice as immutable. +// +// WARNING: Use carefully. The returned result must not leak to the end user +// unless the input slice is provably immutable. +func UnsafeString(b []byte) (s string) { + src := (*sliceHeader)(unsafe.Pointer(&b)) + dst := (*stringHeader)(unsafe.Pointer(&s)) + dst.Data = src.Data + dst.Len = src.Len + return s +} + +// UnsafeBytes returns an unsafe bytes slice reference of s. +// The caller must treat returned slice as immutable. +// +// WARNING: Use carefully. The returned result must not leak to the end user. +func UnsafeBytes(s string) (b []byte) { + src := (*stringHeader)(unsafe.Pointer(&s)) + dst := (*sliceHeader)(unsafe.Pointer(&b)) + dst.Data = src.Data + dst.Len = src.Len + dst.Cap = src.Len + return b +} + +// Builder builds a set of strings with shared lifetime. +// This differs from strings.Builder, which is for building a single string. +type Builder struct { + buf []byte +} + +// AppendFullName is equivalent to protoreflect.FullName.Append, +// but optimized for large batches where each name has a shared lifetime. +func (sb *Builder) AppendFullName(prefix protoreflect.FullName, name protoreflect.Name) protoreflect.FullName { + n := len(prefix) + len(".") + len(name) + if len(prefix) == 0 { + n -= len(".") + } + sb.grow(n) + sb.buf = append(sb.buf, prefix...) + sb.buf = append(sb.buf, '.') + sb.buf = append(sb.buf, name...) + return protoreflect.FullName(sb.last(n)) +} + +// MakeString is equivalent to string(b), but optimized for large batches +// with a shared lifetime. +func (sb *Builder) MakeString(b []byte) string { + sb.grow(len(b)) + sb.buf = append(sb.buf, b...) + return sb.last(len(b)) +} + +func (sb *Builder) grow(n int) { + if cap(sb.buf)-len(sb.buf) >= n { + return + } + + // Unlike strings.Builder, we do not need to copy over the contents + // of the old buffer since our builder provides no API for + // retrieving previously created strings. + sb.buf = make([]byte, 0, 2*(cap(sb.buf)+n)) +} + +func (sb *Builder) last(n int) string { + return UnsafeString(sb.buf[len(sb.buf)-n:]) +} diff --git a/vendor/go.temporal.io/api/internal/protojson/well_known_types.go b/vendor/go.temporal.io/api/internal/protojson/well_known_types.go new file mode 100644 index 00000000000..894630b2ef9 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/protojson/well_known_types.go @@ -0,0 +1,877 @@ +// Copyright 2019 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package protojson + +import ( + "bytes" + "fmt" + "math" + "strconv" + "strings" + "time" + + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/reflect/protoreflect" + + "go.temporal.io/api/internal/protojson/errors" + "go.temporal.io/api/internal/protojson/genid" + "go.temporal.io/api/internal/protojson/json" + "go.temporal.io/api/internal/protojson/strs" +) + +type marshalFunc func(encoder, protoreflect.Message) error + +// wellKnownTypeMarshaler returns a marshal function if the message type +// has specialized serialization behavior. It returns nil otherwise. +func wellKnownTypeMarshaler(name protoreflect.FullName) marshalFunc { + if name.Parent() == genid.GoogleProtobuf_package { + switch name.Name() { + case genid.Any_message_name: + return encoder.marshalAny + case genid.Timestamp_message_name: + return encoder.marshalTimestamp + case genid.Duration_message_name: + return encoder.marshalDuration + case genid.BoolValue_message_name, + genid.Int32Value_message_name, + genid.Int64Value_message_name, + genid.UInt32Value_message_name, + genid.UInt64Value_message_name, + genid.FloatValue_message_name, + genid.DoubleValue_message_name, + genid.StringValue_message_name, + genid.BytesValue_message_name: + return encoder.marshalWrapperType + case genid.Struct_message_name: + return encoder.marshalStruct + case genid.ListValue_message_name: + return encoder.marshalListValue + case genid.Value_message_name: + return encoder.marshalKnownValue + case genid.FieldMask_message_name: + return encoder.marshalFieldMask + case genid.Empty_message_name: + return encoder.marshalEmpty + } + } + return nil +} + +type unmarshalFunc func(decoder, protoreflect.Message) error + +// wellKnownTypeUnmarshaler returns a unmarshal function if the message type +// has specialized serialization behavior. It returns nil otherwise. +func wellKnownTypeUnmarshaler(name protoreflect.FullName) unmarshalFunc { + if name.Parent() == genid.GoogleProtobuf_package { + switch name.Name() { + case genid.Any_message_name: + return decoder.unmarshalAny + case genid.Timestamp_message_name: + return decoder.unmarshalTimestamp + case genid.Duration_message_name: + return decoder.unmarshalDuration + case genid.BoolValue_message_name, + genid.Int32Value_message_name, + genid.Int64Value_message_name, + genid.UInt32Value_message_name, + genid.UInt64Value_message_name, + genid.FloatValue_message_name, + genid.DoubleValue_message_name, + genid.StringValue_message_name, + genid.BytesValue_message_name: + return decoder.unmarshalWrapperType + case genid.Struct_message_name: + return decoder.unmarshalStruct + case genid.ListValue_message_name: + return decoder.unmarshalListValue + case genid.Value_message_name: + return decoder.unmarshalKnownValue + case genid.FieldMask_message_name: + return decoder.unmarshalFieldMask + case genid.Empty_message_name: + return decoder.unmarshalEmpty + } + } + return nil +} + +// The JSON representation of an Any message uses the regular representation of +// the deserialized, embedded message, with an additional field `@type` which +// contains the type URL. If the embedded message type is well-known and has a +// custom JSON representation, that representation will be embedded adding a +// field `value` which holds the custom JSON in addition to the `@type` field. + +func (e encoder) marshalAny(m protoreflect.Message) error { + fds := m.Descriptor().Fields() + fdType := fds.ByNumber(genid.Any_TypeUrl_field_number) + fdValue := fds.ByNumber(genid.Any_Value_field_number) + + if !m.Has(fdType) { + if !m.Has(fdValue) { + // If message is empty, marshal out empty JSON object. + e.StartObject() + e.EndObject() + return nil + } else { + // Return error if type_url field is not set, but value is set. + return errors.New("%s: %v is not set", genid.Any_message_fullname, genid.Any_TypeUrl_field_name) + } + } + + typeVal := m.Get(fdType) + valueVal := m.Get(fdValue) + + // Resolve the type in order to unmarshal value field. + typeURL := typeVal.String() + emt, err := e.opts.Resolver.FindMessageByURL(typeURL) + if err != nil { + return errors.New("%s: unable to resolve %q: %v", genid.Any_message_fullname, typeURL, err) + } + + em := emt.New() + err = proto.UnmarshalOptions{ + AllowPartial: true, // never check required fields inside an Any + Resolver: e.opts.Resolver, + }.Unmarshal(valueVal.Bytes(), em.Interface()) + if err != nil { + return errors.New("%s: unable to unmarshal %q: %v", genid.Any_message_fullname, typeURL, err) + } + + // If type of value has custom JSON encoding, marshal out a field "value" + // with corresponding custom JSON encoding of the embedded message as a + // field. + if marshal := wellKnownTypeMarshaler(emt.Descriptor().FullName()); marshal != nil { + e.StartObject() + defer e.EndObject() + + // Marshal out @type field. + e.WriteName("@type") + if err := e.WriteString(typeURL); err != nil { + return err + } + + e.WriteName("value") + return marshal(e, em) + } + + // Else, marshal out the embedded message's fields in this Any object. + if err := e.marshalMessage(em, typeURL); err != nil { + return err + } + + return nil +} + +func (d decoder) unmarshalAny(m protoreflect.Message) error { + // Peek to check for json.ObjectOpen to avoid advancing a read. + start, err := d.Peek() + if err != nil { + return err + } + if start.Kind() != json.ObjectOpen { + return d.unexpectedTokenError(start) + } + + // Use another decoder to parse the unread bytes for @type field. This + // avoids advancing a read from current decoder because the current JSON + // object may contain the fields of the embedded type. + dec := decoder{d.Clone(), UnmarshalOptions{RecursionLimit: d.opts.RecursionLimit}} + tok, err := findTypeURL(dec) + switch err { + case errEmptyObject: + // An empty JSON object translates to an empty Any message. + d.Read() // Read json.ObjectOpen. + d.Read() // Read json.ObjectClose. + return nil + + case errMissingType: + if d.opts.DiscardUnknown { + // Treat all fields as unknowns, similar to an empty object. + return d.skipJSONValue() + } + // Use start.Pos() for line position. + return d.newError(start.Pos(), err.Error()) + + default: + if err != nil { + return err + } + } + + typeURL := tok.ParsedString() + emt, err := d.opts.Resolver.FindMessageByURL(typeURL) + if err != nil { + return d.newError(tok.Pos(), "unable to resolve %v: %q", tok.RawString(), err) + } + + // Create new message for the embedded message type and unmarshal into it. + em := emt.New() + if unmarshal := wellKnownTypeUnmarshaler(emt.Descriptor().FullName()); unmarshal != nil { + // If embedded message is a custom type, + // unmarshal the JSON "value" field into it. + if err := d.unmarshalAnyValue(unmarshal, em); err != nil { + return err + } + } else { + // Else unmarshal the current JSON object into it. + if err := d.unmarshalMessage(em, true); err != nil { + return err + } + } + // Serialize the embedded message and assign the resulting bytes to the + // proto value field. + b, err := proto.MarshalOptions{ + AllowPartial: true, // No need to check required fields inside an Any. + Deterministic: true, + }.Marshal(em.Interface()) + if err != nil { + return d.newError(start.Pos(), "error in marshaling Any.value field: %v", err) + } + + fds := m.Descriptor().Fields() + fdType := fds.ByNumber(genid.Any_TypeUrl_field_number) + fdValue := fds.ByNumber(genid.Any_Value_field_number) + + m.Set(fdType, protoreflect.ValueOfString(typeURL)) + m.Set(fdValue, protoreflect.ValueOfBytes(b)) + return nil +} + +var errEmptyObject = fmt.Errorf(`empty object`) +var errMissingType = fmt.Errorf(`missing "@type" field`) + +// findTypeURL returns the token for the "@type" field value from the given +// JSON bytes. It is expected that the given bytes start with json.ObjectOpen. +// It returns errEmptyObject if the JSON object is empty or errMissingType if +// @type field does not exist. It returns other error if the @type field is not +// valid or other decoding issues. +func findTypeURL(d decoder) (json.Token, error) { + var typeURL string + var typeTok json.Token + numFields := 0 + // Skip start object. + d.Read() + +Loop: + for { + tok, err := d.Read() + if err != nil { + return json.Token{}, err + } + + switch tok.Kind() { + case json.ObjectClose: + if typeURL == "" { + // Did not find @type field. + if numFields > 0 { + return json.Token{}, errMissingType + } + return json.Token{}, errEmptyObject + } + break Loop + + case json.Name: + numFields++ + if tok.Name() != "@type" { + // Skip value. + if err := d.skipJSONValue(); err != nil { + return json.Token{}, err + } + continue + } + + // Return error if this was previously set already. + if typeURL != "" { + return json.Token{}, d.newError(tok.Pos(), `duplicate "@type" field`) + } + // Read field value. + tok, err := d.Read() + if err != nil { + return json.Token{}, err + } + if tok.Kind() != json.String { + return json.Token{}, d.newError(tok.Pos(), `@type field value is not a string: %v`, tok.RawString()) + } + typeURL = tok.ParsedString() + if typeURL == "" { + return json.Token{}, d.newError(tok.Pos(), `@type field contains empty value`) + } + typeTok = tok + } + } + + return typeTok, nil +} + +// skipJSONValue parses a JSON value (null, boolean, string, number, object and +// array) in order to advance the read to the next JSON value. It relies on +// the decoder returning an error if the types are not in valid sequence. +func (d decoder) skipJSONValue() error { + var open int + for { + tok, err := d.Read() + if err != nil { + return err + } + switch tok.Kind() { + case json.ObjectClose, json.ArrayClose: + open-- + case json.ObjectOpen, json.ArrayOpen: + open++ + if open > d.opts.RecursionLimit { + return errors.New("exceeded max recursion depth") + } + case json.EOF: + // This can only happen if there's a bug in Decoder.Read. + // Avoid an infinite loop if this does happen. + return errors.New("unexpected EOF") + } + if open == 0 { + return nil + } + } +} + +// unmarshalAnyValue unmarshals the given custom-type message from the JSON +// object's "value" field. +func (d decoder) unmarshalAnyValue(unmarshal unmarshalFunc, m protoreflect.Message) error { + // Skip ObjectOpen, and start reading the fields. + d.Read() + + var found bool // Used for detecting duplicate "value". + for { + tok, err := d.Read() + if err != nil { + return err + } + switch tok.Kind() { + case json.ObjectClose: + if !found { + return d.newError(tok.Pos(), `missing "value" field`) + } + return nil + + case json.Name: + switch tok.Name() { + case "@type": + // Skip the value as this was previously parsed already. + d.Read() + + case "value": + if found { + return d.newError(tok.Pos(), `duplicate "value" field`) + } + // Unmarshal the field value into the given message. + if err := unmarshal(d, m); err != nil { + return err + } + found = true + + default: + if d.opts.DiscardUnknown { + if err := d.skipJSONValue(); err != nil { + return err + } + continue + } + return d.newError(tok.Pos(), "unknown field %v", tok.RawString()) + } + } + } +} + +// Wrapper types are encoded as JSON primitives like string, number or boolean. + +func (e encoder) marshalWrapperType(m protoreflect.Message) error { + fd := m.Descriptor().Fields().ByNumber(genid.WrapperValue_Value_field_number) + val := m.Get(fd) + return e.marshalSingular(val, fd) +} + +func (d decoder) unmarshalWrapperType(m protoreflect.Message) error { + fd := m.Descriptor().Fields().ByNumber(genid.WrapperValue_Value_field_number) + val, err := d.unmarshalScalar(fd) + if err != nil { + return err + } + m.Set(fd, val) + return nil +} + +// The JSON representation for Empty is an empty JSON object. + +func (e encoder) marshalEmpty(protoreflect.Message) error { + e.StartObject() + e.EndObject() + return nil +} + +func (d decoder) unmarshalEmpty(protoreflect.Message) error { + tok, err := d.Read() + if err != nil { + return err + } + if tok.Kind() != json.ObjectOpen { + return d.unexpectedTokenError(tok) + } + + for { + tok, err := d.Read() + if err != nil { + return err + } + switch tok.Kind() { + case json.ObjectClose: + return nil + + case json.Name: + if d.opts.DiscardUnknown { + if err := d.skipJSONValue(); err != nil { + return err + } + continue + } + return d.newError(tok.Pos(), "unknown field %v", tok.RawString()) + + default: + return d.unexpectedTokenError(tok) + } + } +} + +// The JSON representation for Struct is a JSON object that contains the encoded +// Struct.fields map and follows the serialization rules for a map. + +func (e encoder) marshalStruct(m protoreflect.Message) error { + fd := m.Descriptor().Fields().ByNumber(genid.Struct_Fields_field_number) + return e.marshalMap(m.Get(fd).Map(), fd) +} + +func (d decoder) unmarshalStruct(m protoreflect.Message) error { + fd := m.Descriptor().Fields().ByNumber(genid.Struct_Fields_field_number) + return d.unmarshalMap(m.Mutable(fd).Map(), fd) +} + +// The JSON representation for ListValue is JSON array that contains the encoded +// ListValue.values repeated field and follows the serialization rules for a +// repeated field. + +func (e encoder) marshalListValue(m protoreflect.Message) error { + fd := m.Descriptor().Fields().ByNumber(genid.ListValue_Values_field_number) + return e.marshalList(m.Get(fd).List(), fd) +} + +func (d decoder) unmarshalListValue(m protoreflect.Message) error { + fd := m.Descriptor().Fields().ByNumber(genid.ListValue_Values_field_number) + return d.unmarshalList(m.Mutable(fd).List(), fd) +} + +// The JSON representation for a Value is dependent on the oneof field that is +// set. Each of the field in the oneof has its own custom serialization rule. A +// Value message needs to be a oneof field set, else it is an error. + +func (e encoder) marshalKnownValue(m protoreflect.Message) error { + od := m.Descriptor().Oneofs().ByName(genid.Value_Kind_oneof_name) + fd := m.WhichOneof(od) + if fd == nil { + return errors.New("%s: none of the oneof fields is set", genid.Value_message_fullname) + } + if fd.Number() == genid.Value_NumberValue_field_number { + if v := m.Get(fd).Float(); math.IsNaN(v) || math.IsInf(v, 0) { + return errors.New("%s: invalid %v value", genid.Value_NumberValue_field_fullname, v) + } + } + return e.marshalSingular(m.Get(fd), fd) +} + +func (d decoder) unmarshalKnownValue(m protoreflect.Message) error { + tok, err := d.Peek() + if err != nil { + return err + } + + var fd protoreflect.FieldDescriptor + var val protoreflect.Value + switch tok.Kind() { + case json.Null: + d.Read() + fd = m.Descriptor().Fields().ByNumber(genid.Value_NullValue_field_number) + val = protoreflect.ValueOfEnum(0) + + case json.Bool: + tok, err := d.Read() + if err != nil { + return err + } + fd = m.Descriptor().Fields().ByNumber(genid.Value_BoolValue_field_number) + val = protoreflect.ValueOfBool(tok.Bool()) + + case json.Number: + tok, err := d.Read() + if err != nil { + return err + } + fd = m.Descriptor().Fields().ByNumber(genid.Value_NumberValue_field_number) + var ok bool + val, ok = unmarshalFloat(tok, 64) + if !ok { + return d.newError(tok.Pos(), "invalid %v: %v", genid.Value_message_fullname, tok.RawString()) + } + + case json.String: + // A JSON string may have been encoded from the number_value field, + // e.g. "NaN", "Infinity", etc. Parsing a proto double type also allows + // for it to be in JSON string form. Given this custom encoding spec, + // however, there is no way to identify that and hence a JSON string is + // always assigned to the string_value field, which means that certain + // encoding cannot be parsed back to the same field. + tok, err := d.Read() + if err != nil { + return err + } + fd = m.Descriptor().Fields().ByNumber(genid.Value_StringValue_field_number) + val = protoreflect.ValueOfString(tok.ParsedString()) + + case json.ObjectOpen: + fd = m.Descriptor().Fields().ByNumber(genid.Value_StructValue_field_number) + val = m.NewField(fd) + if err := d.unmarshalStruct(val.Message()); err != nil { + return err + } + + case json.ArrayOpen: + fd = m.Descriptor().Fields().ByNumber(genid.Value_ListValue_field_number) + val = m.NewField(fd) + if err := d.unmarshalListValue(val.Message()); err != nil { + return err + } + + default: + return d.newError(tok.Pos(), "invalid %v: %v", genid.Value_message_fullname, tok.RawString()) + } + + m.Set(fd, val) + return nil +} + +// The JSON representation for a Duration is a JSON string that ends in the +// suffix "s" (indicating seconds) and is preceded by the number of seconds, +// with nanoseconds expressed as fractional seconds. +// +// Durations less than one second are represented with a 0 seconds field and a +// positive or negative nanos field. For durations of one second or more, a +// non-zero value for the nanos field must be of the same sign as the seconds +// field. +// +// Duration.seconds must be from -315,576,000,000 to +315,576,000,000 inclusive. +// Duration.nanos must be from -999,999,999 to +999,999,999 inclusive. + +const ( + secondsInNanos = 999999999 + maxSecondsInDuration = 315576000000 +) + +func (e encoder) marshalDuration(m protoreflect.Message) error { + fds := m.Descriptor().Fields() + fdSeconds := fds.ByNumber(genid.Duration_Seconds_field_number) + fdNanos := fds.ByNumber(genid.Duration_Nanos_field_number) + + secsVal := m.Get(fdSeconds) + nanosVal := m.Get(fdNanos) + secs := secsVal.Int() + nanos := nanosVal.Int() + if secs < -maxSecondsInDuration || secs > maxSecondsInDuration { + return errors.New("%s: seconds out of range %v", genid.Duration_message_fullname, secs) + } + if nanos < -secondsInNanos || nanos > secondsInNanos { + return errors.New("%s: nanos out of range %v", genid.Duration_message_fullname, nanos) + } + if (secs > 0 && nanos < 0) || (secs < 0 && nanos > 0) { + return errors.New("%s: signs of seconds and nanos do not match", genid.Duration_message_fullname) + } + // Generated output always contains 0, 3, 6, or 9 fractional digits, + // depending on required precision, followed by the suffix "s". + var sign string + if secs < 0 || nanos < 0 { + sign, secs, nanos = "-", -1*secs, -1*nanos + } + x := fmt.Sprintf("%s%d.%09d", sign, secs, nanos) + x = strings.TrimSuffix(x, "000") + x = strings.TrimSuffix(x, "000") + x = strings.TrimSuffix(x, ".000") + e.WriteString(x + "s") + return nil +} + +func (d decoder) unmarshalDuration(m protoreflect.Message) error { + tok, err := d.Read() + if err != nil { + return err + } + if tok.Kind() != json.String { + return d.unexpectedTokenError(tok) + } + + secs, nanos, ok := parseDuration(tok.ParsedString()) + if !ok { + return d.newError(tok.Pos(), "invalid %v value %v", genid.Duration_message_fullname, tok.RawString()) + } + // Validate seconds. No need to validate nanos because parseDuration would + // have covered that already. + if secs < -maxSecondsInDuration || secs > maxSecondsInDuration { + return d.newError(tok.Pos(), "%v value out of range: %v", genid.Duration_message_fullname, tok.RawString()) + } + + fds := m.Descriptor().Fields() + fdSeconds := fds.ByNumber(genid.Duration_Seconds_field_number) + fdNanos := fds.ByNumber(genid.Duration_Nanos_field_number) + + m.Set(fdSeconds, protoreflect.ValueOfInt64(secs)) + m.Set(fdNanos, protoreflect.ValueOfInt32(nanos)) + return nil +} + +// parseDuration parses the given input string for seconds and nanoseconds value +// for the Duration JSON format. The format is a decimal number with a suffix +// 's'. It can have optional plus/minus sign. There needs to be at least an +// integer or fractional part. Fractional part is limited to 9 digits only for +// nanoseconds precision, regardless of whether there are trailing zero digits. +// Example values are 1s, 0.1s, 1.s, .1s, +1s, -1s, -.1s. +func parseDuration(input string) (int64, int32, bool) { + b := []byte(input) + size := len(b) + if size < 2 { + return 0, 0, false + } + if b[size-1] != 's' { + return 0, 0, false + } + b = b[:size-1] + + // Read optional plus/minus symbol. + var neg bool + switch b[0] { + case '-': + neg = true + b = b[1:] + case '+': + b = b[1:] + } + if len(b) == 0 { + return 0, 0, false + } + + // Read the integer part. + var intp []byte + switch { + case b[0] == '0': + b = b[1:] + + case '1' <= b[0] && b[0] <= '9': + intp = b[0:] + b = b[1:] + n := 1 + for len(b) > 0 && '0' <= b[0] && b[0] <= '9' { + n++ + b = b[1:] + } + intp = intp[:n] + + case b[0] == '.': + // Continue below. + + default: + return 0, 0, false + } + + hasFrac := false + var frac [9]byte + if len(b) > 0 { + if b[0] != '.' { + return 0, 0, false + } + // Read the fractional part. + b = b[1:] + n := 0 + for len(b) > 0 && n < 9 && '0' <= b[0] && b[0] <= '9' { + frac[n] = b[0] + n++ + b = b[1:] + } + // It is not valid if there are more bytes left. + if len(b) > 0 { + return 0, 0, false + } + // Pad fractional part with 0s. + for i := n; i < 9; i++ { + frac[i] = '0' + } + hasFrac = true + } + + var secs int64 + if len(intp) > 0 { + var err error + secs, err = strconv.ParseInt(string(intp), 10, 64) + if err != nil { + return 0, 0, false + } + } + + var nanos int64 + if hasFrac { + nanob := bytes.TrimLeft(frac[:], "0") + if len(nanob) > 0 { + var err error + nanos, err = strconv.ParseInt(string(nanob), 10, 32) + if err != nil { + return 0, 0, false + } + } + } + + if neg { + if secs > 0 { + secs = -secs + } + if nanos > 0 { + nanos = -nanos + } + } + return secs, int32(nanos), true +} + +// The JSON representation for a Timestamp is a JSON string in the RFC 3339 +// format, i.e. "{year}-{month}-{day}T{hour}:{min}:{sec}[.{frac_sec}]Z" where +// {year} is always expressed using four digits while {month}, {day}, {hour}, +// {min}, and {sec} are zero-padded to two digits each. The fractional seconds, +// which can go up to 9 digits, up to 1 nanosecond resolution, is optional. The +// "Z" suffix indicates the timezone ("UTC"); the timezone is required. Encoding +// should always use UTC (as indicated by "Z") and a decoder should be able to +// accept both UTC and other timezones (as indicated by an offset). +// +// Timestamp.seconds must be from 0001-01-01T00:00:00Z to 9999-12-31T23:59:59Z +// inclusive. +// Timestamp.nanos must be from 0 to 999,999,999 inclusive. + +const ( + maxTimestampSeconds = 253402300799 + minTimestampSeconds = -62135596800 +) + +func (e encoder) marshalTimestamp(m protoreflect.Message) error { + fds := m.Descriptor().Fields() + fdSeconds := fds.ByNumber(genid.Timestamp_Seconds_field_number) + fdNanos := fds.ByNumber(genid.Timestamp_Nanos_field_number) + + secsVal := m.Get(fdSeconds) + nanosVal := m.Get(fdNanos) + secs := secsVal.Int() + nanos := nanosVal.Int() + if secs < minTimestampSeconds || secs > maxTimestampSeconds { + return errors.New("%s: seconds out of range %v", genid.Timestamp_message_fullname, secs) + } + if nanos < 0 || nanos > secondsInNanos { + return errors.New("%s: nanos out of range %v", genid.Timestamp_message_fullname, nanos) + } + // Uses RFC 3339, where generated output will be Z-normalized and uses 0, 3, + // 6 or 9 fractional digits. + t := time.Unix(secs, nanos).UTC() + x := t.Format("2006-01-02T15:04:05.000000000") + x = strings.TrimSuffix(x, "000") + x = strings.TrimSuffix(x, "000") + x = strings.TrimSuffix(x, ".000") + e.WriteString(x + "Z") + return nil +} + +func (d decoder) unmarshalTimestamp(m protoreflect.Message) error { + tok, err := d.Read() + if err != nil { + return err + } + if tok.Kind() != json.String { + return d.unexpectedTokenError(tok) + } + + s := tok.ParsedString() + t, err := time.Parse(time.RFC3339Nano, s) + if err != nil { + return d.newError(tok.Pos(), "invalid %v value %v", genid.Timestamp_message_fullname, tok.RawString()) + } + // Validate seconds. + secs := t.Unix() + if secs < minTimestampSeconds || secs > maxTimestampSeconds { + return d.newError(tok.Pos(), "%v value out of range: %v", genid.Timestamp_message_fullname, tok.RawString()) + } + // Validate subseconds. + i := strings.LastIndexByte(s, '.') // start of subsecond field + j := strings.LastIndexAny(s, "Z-+") // start of timezone field + if i >= 0 && j >= i && j-i > len(".999999999") { + return d.newError(tok.Pos(), "invalid %v value %v", genid.Timestamp_message_fullname, tok.RawString()) + } + + fds := m.Descriptor().Fields() + fdSeconds := fds.ByNumber(genid.Timestamp_Seconds_field_number) + fdNanos := fds.ByNumber(genid.Timestamp_Nanos_field_number) + + m.Set(fdSeconds, protoreflect.ValueOfInt64(secs)) + m.Set(fdNanos, protoreflect.ValueOfInt32(int32(t.Nanosecond()))) + return nil +} + +// The JSON representation for a FieldMask is a JSON string where paths are +// separated by a comma. Fields name in each path are converted to/from +// lower-camel naming conventions. Encoding should fail if the path name would +// end up differently after a round-trip. + +func (e encoder) marshalFieldMask(m protoreflect.Message) error { + fd := m.Descriptor().Fields().ByNumber(genid.FieldMask_Paths_field_number) + list := m.Get(fd).List() + paths := make([]string, 0, list.Len()) + + for i := 0; i < list.Len(); i++ { + s := list.Get(i).String() + if !protoreflect.FullName(s).IsValid() { + return errors.New("%s contains invalid path: %q", genid.FieldMask_Paths_field_fullname, s) + } + // Return error if conversion to camelCase is not reversible. + cc := strs.JSONCamelCase(s) + if s != strs.JSONSnakeCase(cc) { + return errors.New("%s contains irreversible value %q", genid.FieldMask_Paths_field_fullname, s) + } + paths = append(paths, cc) + } + + e.WriteString(strings.Join(paths, ",")) + return nil +} + +func (d decoder) unmarshalFieldMask(m protoreflect.Message) error { + tok, err := d.Read() + if err != nil { + return err + } + if tok.Kind() != json.String { + return d.unexpectedTokenError(tok) + } + str := strings.TrimSpace(tok.ParsedString()) + if str == "" { + return nil + } + paths := strings.Split(str, ",") + + fd := m.Descriptor().Fields().ByNumber(genid.FieldMask_Paths_field_number) + list := m.Mutable(fd).List() + + for _, s0 := range paths { + s := strs.JSONSnakeCase(s0) + if strings.Contains(s0, "_") || !protoreflect.FullName(s).IsValid() { + return d.newError(tok.Pos(), "%v contains invalid path: %q", genid.FieldMask_Paths_field_fullname, s0) + } + list.Append(protoreflect.ValueOfString(s)) + } + return nil +} diff --git a/vendor/go.temporal.io/api/internal/strcase/strcase.go b/vendor/go.temporal.io/api/internal/strcase/strcase.go new file mode 100644 index 00000000000..8a10c033817 --- /dev/null +++ b/vendor/go.temporal.io/api/internal/strcase/strcase.go @@ -0,0 +1,139 @@ +// The MIT License +// +// Copyright (c) 2015 Ian Coleman +// Copyright (c) 2018 Ma_124, +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Package strcase copies the portions of code we need from https://github.com/iancoleman/strcase. This is done to avoid +// adding a new dependency to the project since many of the functions in strcase are not used, and api-go is used by so +// many other projects. +package strcase + +import ( + "strings" +) + +// ToScreamingSnake converts a string to SCREAMING_SNAKE_CASE +func ToScreamingSnake(s string) string { + return ToScreamingDelimited(s, '_', "", true) +} + +// ToScreamingDelimited converts a string to SCREAMING.DELIMITED.SNAKE.CASE +// (in this case `delimiter = '.'; screaming = true`) +// or delimited.snake.case +// (in this case `delimiter = '.'; screaming = false`) +func ToScreamingDelimited(s string, delimiter uint8, ignore string, screaming bool) string { + s = strings.TrimSpace(s) + n := strings.Builder{} + n.Grow(len(s) + 2) // nominal 2 bytes of extra space for inserted delimiters + for i, v := range []byte(s) { + vIsCap := v >= 'A' && v <= 'Z' + vIsLow := v >= 'a' && v <= 'z' + if vIsLow && screaming { + v += 'A' + v -= 'a' + } else if vIsCap && !screaming { + v += 'a' + v -= 'A' + } + + // treat acronyms as words, eg for JSONData -> JSON is a whole word + if i+1 < len(s) { + next := s[i+1] + vIsNum := v >= '0' && v <= '9' + nextIsCap := next >= 'A' && next <= 'Z' + nextIsLow := next >= 'a' && next <= 'z' + nextIsNum := next >= '0' && next <= '9' + // add underscore if next letter case type is changed + if (vIsCap && (nextIsLow || nextIsNum)) || (vIsLow && (nextIsCap || nextIsNum)) || (vIsNum && (nextIsCap || nextIsLow)) { + prevIgnore := ignore != "" && i > 0 && strings.ContainsAny(string(s[i-1]), ignore) + if !prevIgnore { + if vIsCap && nextIsLow { + if prevIsCap := i > 0 && s[i-1] >= 'A' && s[i-1] <= 'Z'; prevIsCap { + n.WriteByte(delimiter) + } + } + n.WriteByte(v) + if vIsLow || vIsNum || nextIsNum { + n.WriteByte(delimiter) + } + continue + } + } + } + + if (v == ' ' || v == '_' || v == '-' || v == '.') && !strings.ContainsAny(string(v), ignore) { + // replace space/underscore/hyphen/dot with delimiter + n.WriteByte(delimiter) + } else { + n.WriteByte(v) + } + } + + return n.String() +} + +// ToCamel converts a string to CamelCase +func ToCamel(s string) string { + return toCamelInitCase(s, true) +} + +// Converts a string to CamelCase +func toCamelInitCase(s string, initCase bool) string { + s = strings.TrimSpace(s) + if s == "" { + return s + } + + n := strings.Builder{} + n.Grow(len(s)) + capNext := initCase + prevIsCap := false + for i, v := range []byte(s) { + vIsCap := v >= 'A' && v <= 'Z' + vIsLow := v >= 'a' && v <= 'z' + if capNext { + if vIsLow { + v += 'A' + v -= 'a' + } + } else if i == 0 { + if vIsCap { + v += 'a' + v -= 'A' + } + } else if prevIsCap && vIsCap { + v += 'a' + v -= 'A' + } + prevIsCap = vIsCap + + if vIsCap || vIsLow { + n.WriteByte(v) + capNext = false + } else if vIsNum := v >= '0' && v <= '9'; vIsNum { + n.WriteByte(v) + capNext = true + } else { + capNext = v == '_' || v == ' ' || v == '-' || v == '.' + } + } + return n.String() +} diff --git a/vendor/go.temporal.io/api/namespace/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/namespace/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..e552c16c10a --- /dev/null +++ b/vendor/go.temporal.io/api/namespace/v1/message.go-helpers.pb.go @@ -0,0 +1,250 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package namespace + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type NamespaceInfo to the protobuf v3 wire format +func (val *NamespaceInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NamespaceInfo from the protobuf v3 wire format +func (val *NamespaceInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NamespaceInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NamespaceInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NamespaceInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NamespaceInfo + switch t := that.(type) { + case *NamespaceInfo: + that1 = t + case NamespaceInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NamespaceConfig to the protobuf v3 wire format +func (val *NamespaceConfig) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NamespaceConfig from the protobuf v3 wire format +func (val *NamespaceConfig) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NamespaceConfig) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NamespaceConfig values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NamespaceConfig) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NamespaceConfig + switch t := that.(type) { + case *NamespaceConfig: + that1 = t + case NamespaceConfig: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type BadBinaries to the protobuf v3 wire format +func (val *BadBinaries) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type BadBinaries from the protobuf v3 wire format +func (val *BadBinaries) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *BadBinaries) Size() int { + return proto.Size(val) +} + +// Equal returns whether two BadBinaries values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *BadBinaries) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *BadBinaries + switch t := that.(type) { + case *BadBinaries: + that1 = t + case BadBinaries: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type BadBinaryInfo to the protobuf v3 wire format +func (val *BadBinaryInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type BadBinaryInfo from the protobuf v3 wire format +func (val *BadBinaryInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *BadBinaryInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two BadBinaryInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *BadBinaryInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *BadBinaryInfo + switch t := that.(type) { + case *BadBinaryInfo: + that1 = t + case BadBinaryInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateNamespaceInfo to the protobuf v3 wire format +func (val *UpdateNamespaceInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateNamespaceInfo from the protobuf v3 wire format +func (val *UpdateNamespaceInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateNamespaceInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateNamespaceInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateNamespaceInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateNamespaceInfo + switch t := that.(type) { + case *UpdateNamespaceInfo: + that1 = t + case UpdateNamespaceInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NamespaceFilter to the protobuf v3 wire format +func (val *NamespaceFilter) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NamespaceFilter from the protobuf v3 wire format +func (val *NamespaceFilter) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NamespaceFilter) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NamespaceFilter values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NamespaceFilter) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NamespaceFilter + switch t := that.(type) { + case *NamespaceFilter: + that1 = t + case NamespaceFilter: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/namespace/v1/message.pb.go b/vendor/go.temporal.io/api/namespace/v1/message.pb.go new file mode 100644 index 00000000000..71a27beafce --- /dev/null +++ b/vendor/go.temporal.io/api/namespace/v1/message.pb.go @@ -0,0 +1,872 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/namespace/v1/message.proto + +package namespace + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/enums/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + durationpb "google.golang.org/protobuf/types/known/durationpb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type NamespaceInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + State v1.NamespaceState `protobuf:"varint,2,opt,name=state,proto3,enum=temporal.api.enums.v1.NamespaceState" json:"state,omitempty"` + Description string `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"` + OwnerEmail string `protobuf:"bytes,4,opt,name=owner_email,json=ownerEmail,proto3" json:"owner_email,omitempty"` + // A key-value map for any customized purpose. + Data map[string]string `protobuf:"bytes,5,rep,name=data,proto3" json:"data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Id string `protobuf:"bytes,6,opt,name=id,proto3" json:"id,omitempty"` + // All capabilities the namespace supports. + Capabilities *NamespaceInfo_Capabilities `protobuf:"bytes,7,opt,name=capabilities,proto3" json:"capabilities,omitempty"` + // Whether scheduled workflows are supported on this namespace. This is only needed + // temporarily while the feature is experimental, so we can give it a high tag. + SupportsSchedules bool `protobuf:"varint,100,opt,name=supports_schedules,json=supportsSchedules,proto3" json:"supports_schedules,omitempty"` +} + +func (x *NamespaceInfo) Reset() { + *x = NamespaceInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NamespaceInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NamespaceInfo) ProtoMessage() {} + +func (x *NamespaceInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NamespaceInfo.ProtoReflect.Descriptor instead. +func (*NamespaceInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_namespace_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *NamespaceInfo) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *NamespaceInfo) GetState() v1.NamespaceState { + if x != nil { + return x.State + } + return v1.NamespaceState(0) +} + +func (x *NamespaceInfo) GetDescription() string { + if x != nil { + return x.Description + } + return "" +} + +func (x *NamespaceInfo) GetOwnerEmail() string { + if x != nil { + return x.OwnerEmail + } + return "" +} + +func (x *NamespaceInfo) GetData() map[string]string { + if x != nil { + return x.Data + } + return nil +} + +func (x *NamespaceInfo) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *NamespaceInfo) GetCapabilities() *NamespaceInfo_Capabilities { + if x != nil { + return x.Capabilities + } + return nil +} + +func (x *NamespaceInfo) GetSupportsSchedules() bool { + if x != nil { + return x.SupportsSchedules + } + return false +} + +type NamespaceConfig struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + WorkflowExecutionRetentionTtl *durationpb.Duration `protobuf:"bytes,1,opt,name=workflow_execution_retention_ttl,json=workflowExecutionRetentionTtl,proto3" json:"workflow_execution_retention_ttl,omitempty"` + BadBinaries *BadBinaries `protobuf:"bytes,2,opt,name=bad_binaries,json=badBinaries,proto3" json:"bad_binaries,omitempty"` + // If unspecified (ARCHIVAL_STATE_UNSPECIFIED) then default server configuration is used. + HistoryArchivalState v1.ArchivalState `protobuf:"varint,3,opt,name=history_archival_state,json=historyArchivalState,proto3,enum=temporal.api.enums.v1.ArchivalState" json:"history_archival_state,omitempty"` + HistoryArchivalUri string `protobuf:"bytes,4,opt,name=history_archival_uri,json=historyArchivalUri,proto3" json:"history_archival_uri,omitempty"` + // If unspecified (ARCHIVAL_STATE_UNSPECIFIED) then default server configuration is used. + VisibilityArchivalState v1.ArchivalState `protobuf:"varint,5,opt,name=visibility_archival_state,json=visibilityArchivalState,proto3,enum=temporal.api.enums.v1.ArchivalState" json:"visibility_archival_state,omitempty"` + VisibilityArchivalUri string `protobuf:"bytes,6,opt,name=visibility_archival_uri,json=visibilityArchivalUri,proto3" json:"visibility_archival_uri,omitempty"` + // Map from field name to alias. + CustomSearchAttributeAliases map[string]string `protobuf:"bytes,7,rep,name=custom_search_attribute_aliases,json=customSearchAttributeAliases,proto3" json:"custom_search_attribute_aliases,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *NamespaceConfig) Reset() { + *x = NamespaceConfig{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NamespaceConfig) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NamespaceConfig) ProtoMessage() {} + +func (x *NamespaceConfig) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NamespaceConfig.ProtoReflect.Descriptor instead. +func (*NamespaceConfig) Descriptor() ([]byte, []int) { + return file_temporal_api_namespace_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *NamespaceConfig) GetWorkflowExecutionRetentionTtl() *durationpb.Duration { + if x != nil { + return x.WorkflowExecutionRetentionTtl + } + return nil +} + +func (x *NamespaceConfig) GetBadBinaries() *BadBinaries { + if x != nil { + return x.BadBinaries + } + return nil +} + +func (x *NamespaceConfig) GetHistoryArchivalState() v1.ArchivalState { + if x != nil { + return x.HistoryArchivalState + } + return v1.ArchivalState(0) +} + +func (x *NamespaceConfig) GetHistoryArchivalUri() string { + if x != nil { + return x.HistoryArchivalUri + } + return "" +} + +func (x *NamespaceConfig) GetVisibilityArchivalState() v1.ArchivalState { + if x != nil { + return x.VisibilityArchivalState + } + return v1.ArchivalState(0) +} + +func (x *NamespaceConfig) GetVisibilityArchivalUri() string { + if x != nil { + return x.VisibilityArchivalUri + } + return "" +} + +func (x *NamespaceConfig) GetCustomSearchAttributeAliases() map[string]string { + if x != nil { + return x.CustomSearchAttributeAliases + } + return nil +} + +type BadBinaries struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Binaries map[string]*BadBinaryInfo `protobuf:"bytes,1,rep,name=binaries,proto3" json:"binaries,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *BadBinaries) Reset() { + *x = BadBinaries{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BadBinaries) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BadBinaries) ProtoMessage() {} + +func (x *BadBinaries) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BadBinaries.ProtoReflect.Descriptor instead. +func (*BadBinaries) Descriptor() ([]byte, []int) { + return file_temporal_api_namespace_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *BadBinaries) GetBinaries() map[string]*BadBinaryInfo { + if x != nil { + return x.Binaries + } + return nil +} + +type BadBinaryInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Reason string `protobuf:"bytes,1,opt,name=reason,proto3" json:"reason,omitempty"` + Operator string `protobuf:"bytes,2,opt,name=operator,proto3" json:"operator,omitempty"` + CreateTime *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` +} + +func (x *BadBinaryInfo) Reset() { + *x = BadBinaryInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BadBinaryInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BadBinaryInfo) ProtoMessage() {} + +func (x *BadBinaryInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BadBinaryInfo.ProtoReflect.Descriptor instead. +func (*BadBinaryInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_namespace_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *BadBinaryInfo) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *BadBinaryInfo) GetOperator() string { + if x != nil { + return x.Operator + } + return "" +} + +func (x *BadBinaryInfo) GetCreateTime() *timestamppb.Timestamp { + if x != nil { + return x.CreateTime + } + return nil +} + +type UpdateNamespaceInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Description string `protobuf:"bytes,1,opt,name=description,proto3" json:"description,omitempty"` + OwnerEmail string `protobuf:"bytes,2,opt,name=owner_email,json=ownerEmail,proto3" json:"owner_email,omitempty"` + // A key-value map for any customized purpose. + // If data already exists on the namespace, + // this will merge with the existing key values. + Data map[string]string `protobuf:"bytes,3,rep,name=data,proto3" json:"data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // New namespace state, server will reject if transition is not allowed. + // Allowed transitions are: + // + // Registered -> [ Deleted | Deprecated | Handover ] + // Handover -> [ Registered ] + // + // Default is NAMESPACE_STATE_UNSPECIFIED which is do not change state. + State v1.NamespaceState `protobuf:"varint,4,opt,name=state,proto3,enum=temporal.api.enums.v1.NamespaceState" json:"state,omitempty"` +} + +func (x *UpdateNamespaceInfo) Reset() { + *x = UpdateNamespaceInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateNamespaceInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateNamespaceInfo) ProtoMessage() {} + +func (x *UpdateNamespaceInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateNamespaceInfo.ProtoReflect.Descriptor instead. +func (*UpdateNamespaceInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_namespace_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *UpdateNamespaceInfo) GetDescription() string { + if x != nil { + return x.Description + } + return "" +} + +func (x *UpdateNamespaceInfo) GetOwnerEmail() string { + if x != nil { + return x.OwnerEmail + } + return "" +} + +func (x *UpdateNamespaceInfo) GetData() map[string]string { + if x != nil { + return x.Data + } + return nil +} + +func (x *UpdateNamespaceInfo) GetState() v1.NamespaceState { + if x != nil { + return x.State + } + return v1.NamespaceState(0) +} + +type NamespaceFilter struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // By default namespaces in NAMESPACE_STATE_DELETED state are not included. + // Setting include_deleted to true will include deleted namespaces. + // Note: Namespace is in NAMESPACE_STATE_DELETED state when it was deleted from the system but associated data is not deleted yet. + IncludeDeleted bool `protobuf:"varint,1,opt,name=include_deleted,json=includeDeleted,proto3" json:"include_deleted,omitempty"` +} + +func (x *NamespaceFilter) Reset() { + *x = NamespaceFilter{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NamespaceFilter) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NamespaceFilter) ProtoMessage() {} + +func (x *NamespaceFilter) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NamespaceFilter.ProtoReflect.Descriptor instead. +func (*NamespaceFilter) Descriptor() ([]byte, []int) { + return file_temporal_api_namespace_v1_message_proto_rawDescGZIP(), []int{5} +} + +func (x *NamespaceFilter) GetIncludeDeleted() bool { + if x != nil { + return x.IncludeDeleted + } + return false +} + +// Namespace capability details. Should contain what features are enabled in a namespace. +type NamespaceInfo_Capabilities struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // True if the namespace supports eager workflow start. + EagerWorkflowStart bool `protobuf:"varint,1,opt,name=eager_workflow_start,json=eagerWorkflowStart,proto3" json:"eager_workflow_start,omitempty"` + // True if the namespace supports sync update + SyncUpdate bool `protobuf:"varint,2,opt,name=sync_update,json=syncUpdate,proto3" json:"sync_update,omitempty"` + // True if the namespace supports async update + AsyncUpdate bool `protobuf:"varint,3,opt,name=async_update,json=asyncUpdate,proto3" json:"async_update,omitempty"` +} + +func (x *NamespaceInfo_Capabilities) Reset() { + *x = NamespaceInfo_Capabilities{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NamespaceInfo_Capabilities) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NamespaceInfo_Capabilities) ProtoMessage() {} + +func (x *NamespaceInfo_Capabilities) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_namespace_v1_message_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NamespaceInfo_Capabilities.ProtoReflect.Descriptor instead. +func (*NamespaceInfo_Capabilities) Descriptor() ([]byte, []int) { + return file_temporal_api_namespace_v1_message_proto_rawDescGZIP(), []int{0, 1} +} + +func (x *NamespaceInfo_Capabilities) GetEagerWorkflowStart() bool { + if x != nil { + return x.EagerWorkflowStart + } + return false +} + +func (x *NamespaceInfo_Capabilities) GetSyncUpdate() bool { + if x != nil { + return x.SyncUpdate + } + return false +} + +func (x *NamespaceInfo_Capabilities) GetAsyncUpdate() bool { + if x != nil { + return x.AsyncUpdate + } + return false +} + +var File_temporal_api_namespace_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_namespace_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x27, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x19, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, + 0x1a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, + 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x22, 0xf9, 0x04, 0x0a, 0x0d, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, + 0x6e, 0x66, 0x6f, 0x12, 0x16, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, 0x0a, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x24, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x6f, 0x77, + 0x6e, 0x65, 0x72, 0x5f, 0x65, 0x6d, 0x61, 0x69, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, + 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x45, 0x6d, 0x61, 0x69, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, + 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x6e, + 0x66, 0x6f, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x64, 0x61, 0x74, + 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, 0x12, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x02, 0x69, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5d, 0x0a, 0x0c, 0x63, 0x61, 0x70, 0x61, 0x62, + 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x43, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, + 0x73, 0x52, 0x0c, 0x63, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x31, 0x0a, 0x12, 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x5f, 0x73, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x64, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x73, 0x75, + 0x70, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x1a, 0x3f, 0x0a, 0x09, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x1a, + 0x90, 0x01, 0x0a, 0x0c, 0x43, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x12, + 0x34, 0x0a, 0x14, 0x65, 0x61, 0x67, 0x65, 0x72, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x65, 0x61, 0x67, + 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x72, 0x74, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x73, 0x79, 0x6e, 0x63, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x75, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x61, 0x73, 0x79, 0x6e, 0x63, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xf3, 0x05, 0x0a, 0x0f, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x66, 0x0a, 0x20, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x1d, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x74, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, + 0x0a, 0x0c, 0x62, 0x61, 0x64, 0x5f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x61, + 0x64, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x69, 0x65, 0x73, 0x52, 0x0b, 0x62, 0x61, 0x64, 0x42, 0x69, + 0x6e, 0x61, 0x72, 0x69, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5e, 0x0a, 0x16, 0x68, 0x69, 0x73, + 0x74, 0x6f, 0x72, 0x79, 0x5f, 0x61, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x5f, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x41, + 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x14, 0x68, 0x69, 0x73, + 0x74, 0x6f, 0x72, 0x79, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, 0x14, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x5f, 0x61, + 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x5f, 0x75, 0x72, 0x69, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x12, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, + 0x55, 0x72, 0x69, 0x42, 0x02, 0x68, 0x00, 0x12, 0x64, 0x0a, 0x19, 0x76, 0x69, 0x73, 0x69, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x61, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x5f, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x41, + 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x17, 0x76, 0x69, 0x73, + 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x17, 0x76, 0x69, 0x73, 0x69, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x61, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x5f, 0x75, 0x72, 0x69, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x15, 0x76, 0x69, 0x73, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, + 0x79, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x55, 0x72, 0x69, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x97, 0x01, 0x0a, 0x1f, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, + 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x2e, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x1c, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x1a, 0x57, 0x0a, 0x21, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xd2, 0x01, 0x0a, 0x0b, 0x42, 0x61, 0x64, 0x42, 0x69, 0x6e, 0x61, + 0x72, 0x69, 0x65, 0x73, 0x12, 0x54, 0x0a, 0x08, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x69, 0x65, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x42, 0x61, 0x64, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x69, 0x65, 0x73, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, + 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x69, 0x65, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x6d, 0x0a, 0x0d, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x69, 0x65, 0x73, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x42, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x42, 0x61, 0x64, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x8c, 0x01, + 0x0a, 0x0d, 0x42, 0x61, 0x64, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1a, + 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, + 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x6f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x6f, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x22, 0xb4, 0x02, 0x0a, 0x13, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x24, 0x0a, 0x0b, 0x64, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, + 0x0a, 0x0b, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x5f, 0x65, 0x6d, 0x61, 0x69, 0x6c, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x45, 0x6d, 0x61, 0x69, 0x6c, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x50, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x38, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x44, 0x61, 0x74, + 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x3f, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, + 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x3f, 0x0a, 0x09, + 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3e, 0x0a, 0x0f, 0x4e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x2b, 0x0a, 0x0f, 0x69, + 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x42, 0x98, 0x01, 0x0a, 0x1c, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, + 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x29, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x2f, 0x76, 0x31, 0x3b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0xaa, 0x02, 0x1b, 0x54, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x4e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1e, 0x54, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_namespace_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_namespace_v1_message_proto_rawDescData = file_temporal_api_namespace_v1_message_proto_rawDesc +) + +func file_temporal_api_namespace_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_namespace_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_namespace_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_namespace_v1_message_proto_rawDescData) + }) + return file_temporal_api_namespace_v1_message_proto_rawDescData +} + +var file_temporal_api_namespace_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 11) +var file_temporal_api_namespace_v1_message_proto_goTypes = []any{ + (*NamespaceInfo)(nil), // 0: temporal.api.namespace.v1.NamespaceInfo + (*NamespaceConfig)(nil), // 1: temporal.api.namespace.v1.NamespaceConfig + (*BadBinaries)(nil), // 2: temporal.api.namespace.v1.BadBinaries + (*BadBinaryInfo)(nil), // 3: temporal.api.namespace.v1.BadBinaryInfo + (*UpdateNamespaceInfo)(nil), // 4: temporal.api.namespace.v1.UpdateNamespaceInfo + (*NamespaceFilter)(nil), // 5: temporal.api.namespace.v1.NamespaceFilter + nil, // 6: temporal.api.namespace.v1.NamespaceInfo.DataEntry + (*NamespaceInfo_Capabilities)(nil), // 7: temporal.api.namespace.v1.NamespaceInfo.Capabilities + nil, // 8: temporal.api.namespace.v1.NamespaceConfig.CustomSearchAttributeAliasesEntry + nil, // 9: temporal.api.namespace.v1.BadBinaries.BinariesEntry + nil, // 10: temporal.api.namespace.v1.UpdateNamespaceInfo.DataEntry + (v1.NamespaceState)(0), // 11: temporal.api.enums.v1.NamespaceState + (*durationpb.Duration)(nil), // 12: google.protobuf.Duration + (v1.ArchivalState)(0), // 13: temporal.api.enums.v1.ArchivalState + (*timestamppb.Timestamp)(nil), // 14: google.protobuf.Timestamp +} +var file_temporal_api_namespace_v1_message_proto_depIdxs = []int32{ + 11, // 0: temporal.api.namespace.v1.NamespaceInfo.state:type_name -> temporal.api.enums.v1.NamespaceState + 6, // 1: temporal.api.namespace.v1.NamespaceInfo.data:type_name -> temporal.api.namespace.v1.NamespaceInfo.DataEntry + 7, // 2: temporal.api.namespace.v1.NamespaceInfo.capabilities:type_name -> temporal.api.namespace.v1.NamespaceInfo.Capabilities + 12, // 3: temporal.api.namespace.v1.NamespaceConfig.workflow_execution_retention_ttl:type_name -> google.protobuf.Duration + 2, // 4: temporal.api.namespace.v1.NamespaceConfig.bad_binaries:type_name -> temporal.api.namespace.v1.BadBinaries + 13, // 5: temporal.api.namespace.v1.NamespaceConfig.history_archival_state:type_name -> temporal.api.enums.v1.ArchivalState + 13, // 6: temporal.api.namespace.v1.NamespaceConfig.visibility_archival_state:type_name -> temporal.api.enums.v1.ArchivalState + 8, // 7: temporal.api.namespace.v1.NamespaceConfig.custom_search_attribute_aliases:type_name -> temporal.api.namespace.v1.NamespaceConfig.CustomSearchAttributeAliasesEntry + 9, // 8: temporal.api.namespace.v1.BadBinaries.binaries:type_name -> temporal.api.namespace.v1.BadBinaries.BinariesEntry + 14, // 9: temporal.api.namespace.v1.BadBinaryInfo.create_time:type_name -> google.protobuf.Timestamp + 10, // 10: temporal.api.namespace.v1.UpdateNamespaceInfo.data:type_name -> temporal.api.namespace.v1.UpdateNamespaceInfo.DataEntry + 11, // 11: temporal.api.namespace.v1.UpdateNamespaceInfo.state:type_name -> temporal.api.enums.v1.NamespaceState + 3, // 12: temporal.api.namespace.v1.BadBinaries.BinariesEntry.value:type_name -> temporal.api.namespace.v1.BadBinaryInfo + 13, // [13:13] is the sub-list for method output_type + 13, // [13:13] is the sub-list for method input_type + 13, // [13:13] is the sub-list for extension type_name + 13, // [13:13] is the sub-list for extension extendee + 0, // [0:13] is the sub-list for field type_name +} + +func init() { file_temporal_api_namespace_v1_message_proto_init() } +func file_temporal_api_namespace_v1_message_proto_init() { + if File_temporal_api_namespace_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_namespace_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*NamespaceInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_namespace_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*NamespaceConfig); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_namespace_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*BadBinaries); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_namespace_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*BadBinaryInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_namespace_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*UpdateNamespaceInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_namespace_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*NamespaceFilter); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_namespace_v1_message_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*NamespaceInfo_Capabilities); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_namespace_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 11, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_namespace_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_namespace_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_namespace_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_namespace_v1_message_proto = out.File + file_temporal_api_namespace_v1_message_proto_rawDesc = nil + file_temporal_api_namespace_v1_message_proto_goTypes = nil + file_temporal_api_namespace_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/nexus/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/nexus/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..17183f0d4af --- /dev/null +++ b/vendor/go.temporal.io/api/nexus/v1/message.go-helpers.pb.go @@ -0,0 +1,509 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package nexus + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type Failure to the protobuf v3 wire format +func (val *Failure) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Failure from the protobuf v3 wire format +func (val *Failure) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Failure) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Failure values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Failure) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Failure + switch t := that.(type) { + case *Failure: + that1 = t + case Failure: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type HandlerError to the protobuf v3 wire format +func (val *HandlerError) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type HandlerError from the protobuf v3 wire format +func (val *HandlerError) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *HandlerError) Size() int { + return proto.Size(val) +} + +// Equal returns whether two HandlerError values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *HandlerError) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *HandlerError + switch t := that.(type) { + case *HandlerError: + that1 = t + case HandlerError: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UnsuccessfulOperationError to the protobuf v3 wire format +func (val *UnsuccessfulOperationError) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UnsuccessfulOperationError from the protobuf v3 wire format +func (val *UnsuccessfulOperationError) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UnsuccessfulOperationError) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UnsuccessfulOperationError values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UnsuccessfulOperationError) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UnsuccessfulOperationError + switch t := that.(type) { + case *UnsuccessfulOperationError: + that1 = t + case UnsuccessfulOperationError: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Link to the protobuf v3 wire format +func (val *Link) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Link from the protobuf v3 wire format +func (val *Link) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Link) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Link values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Link) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Link + switch t := that.(type) { + case *Link: + that1 = t + case Link: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StartOperationRequest to the protobuf v3 wire format +func (val *StartOperationRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StartOperationRequest from the protobuf v3 wire format +func (val *StartOperationRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StartOperationRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StartOperationRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StartOperationRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StartOperationRequest + switch t := that.(type) { + case *StartOperationRequest: + that1 = t + case StartOperationRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CancelOperationRequest to the protobuf v3 wire format +func (val *CancelOperationRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CancelOperationRequest from the protobuf v3 wire format +func (val *CancelOperationRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CancelOperationRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CancelOperationRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CancelOperationRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CancelOperationRequest + switch t := that.(type) { + case *CancelOperationRequest: + that1 = t + case CancelOperationRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Request to the protobuf v3 wire format +func (val *Request) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Request from the protobuf v3 wire format +func (val *Request) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Request) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Request values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Request) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Request + switch t := that.(type) { + case *Request: + that1 = t + case Request: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StartOperationResponse to the protobuf v3 wire format +func (val *StartOperationResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StartOperationResponse from the protobuf v3 wire format +func (val *StartOperationResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StartOperationResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StartOperationResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StartOperationResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StartOperationResponse + switch t := that.(type) { + case *StartOperationResponse: + that1 = t + case StartOperationResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CancelOperationResponse to the protobuf v3 wire format +func (val *CancelOperationResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CancelOperationResponse from the protobuf v3 wire format +func (val *CancelOperationResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CancelOperationResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CancelOperationResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CancelOperationResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CancelOperationResponse + switch t := that.(type) { + case *CancelOperationResponse: + that1 = t + case CancelOperationResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Response to the protobuf v3 wire format +func (val *Response) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Response from the protobuf v3 wire format +func (val *Response) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Response) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Response values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Response) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Response + switch t := that.(type) { + case *Response: + that1 = t + case Response: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Endpoint to the protobuf v3 wire format +func (val *Endpoint) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Endpoint from the protobuf v3 wire format +func (val *Endpoint) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Endpoint) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Endpoint values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Endpoint) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Endpoint + switch t := that.(type) { + case *Endpoint: + that1 = t + case Endpoint: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type EndpointSpec to the protobuf v3 wire format +func (val *EndpointSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type EndpointSpec from the protobuf v3 wire format +func (val *EndpointSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *EndpointSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two EndpointSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *EndpointSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *EndpointSpec + switch t := that.(type) { + case *EndpointSpec: + that1 = t + case EndpointSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type EndpointTarget to the protobuf v3 wire format +func (val *EndpointTarget) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type EndpointTarget from the protobuf v3 wire format +func (val *EndpointTarget) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *EndpointTarget) Size() int { + return proto.Size(val) +} + +// Equal returns whether two EndpointTarget values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *EndpointTarget) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *EndpointTarget + switch t := that.(type) { + case *EndpointTarget: + that1 = t + case EndpointTarget: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/nexus/v1/message.pb.go b/vendor/go.temporal.io/api/nexus/v1/message.pb.go new file mode 100644 index 00000000000..9edabeb8908 --- /dev/null +++ b/vendor/go.temporal.io/api/nexus/v1/message.pb.go @@ -0,0 +1,1758 @@ +// The MIT License +// +// Copyright (c) 2023 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/nexus/v1/message.proto + +package nexus + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/common/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// A general purpose failure message. +// See: https://github.com/nexus-rpc/api/blob/main/SPEC.md#failure +type Failure struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Message string `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"` + Metadata map[string]string `protobuf:"bytes,2,rep,name=metadata,proto3" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Details []byte `protobuf:"bytes,3,opt,name=details,proto3" json:"details,omitempty"` +} + +func (x *Failure) Reset() { + *x = Failure{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Failure) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Failure) ProtoMessage() {} + +func (x *Failure) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Failure.ProtoReflect.Descriptor instead. +func (*Failure) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *Failure) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + +func (x *Failure) GetMetadata() map[string]string { + if x != nil { + return x.Metadata + } + return nil +} + +func (x *Failure) GetDetails() []byte { + if x != nil { + return x.Details + } + return nil +} + +type HandlerError struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // See https://github.com/nexus-rpc/api/blob/main/SPEC.md#predefined-handler-errors. + ErrorType string `protobuf:"bytes,1,opt,name=error_type,json=errorType,proto3" json:"error_type,omitempty"` + Failure *Failure `protobuf:"bytes,2,opt,name=failure,proto3" json:"failure,omitempty"` +} + +func (x *HandlerError) Reset() { + *x = HandlerError{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *HandlerError) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HandlerError) ProtoMessage() {} + +func (x *HandlerError) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use HandlerError.ProtoReflect.Descriptor instead. +func (*HandlerError) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *HandlerError) GetErrorType() string { + if x != nil { + return x.ErrorType + } + return "" +} + +func (x *HandlerError) GetFailure() *Failure { + if x != nil { + return x.Failure + } + return nil +} + +type UnsuccessfulOperationError struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // See https://github.com/nexus-rpc/api/blob/main/SPEC.md#operationinfo. + OperationState string `protobuf:"bytes,1,opt,name=operation_state,json=operationState,proto3" json:"operation_state,omitempty"` + Failure *Failure `protobuf:"bytes,2,opt,name=failure,proto3" json:"failure,omitempty"` +} + +func (x *UnsuccessfulOperationError) Reset() { + *x = UnsuccessfulOperationError{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UnsuccessfulOperationError) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UnsuccessfulOperationError) ProtoMessage() {} + +func (x *UnsuccessfulOperationError) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UnsuccessfulOperationError.ProtoReflect.Descriptor instead. +func (*UnsuccessfulOperationError) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *UnsuccessfulOperationError) GetOperationState() string { + if x != nil { + return x.OperationState + } + return "" +} + +func (x *UnsuccessfulOperationError) GetFailure() *Failure { + if x != nil { + return x.Failure + } + return nil +} + +type Link struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // See https://github.com/nexus-rpc/api/blob/main/SPEC.md#links. + Url string `protobuf:"bytes,1,opt,name=url,proto3" json:"url,omitempty"` + Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` +} + +func (x *Link) Reset() { + *x = Link{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Link) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Link) ProtoMessage() {} + +func (x *Link) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Link.ProtoReflect.Descriptor instead. +func (*Link) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *Link) GetUrl() string { + if x != nil { + return x.Url + } + return "" +} + +func (x *Link) GetType() string { + if x != nil { + return x.Type + } + return "" +} + +// A request to start an operation. +type StartOperationRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Name of service to start the operation in. + Service string `protobuf:"bytes,1,opt,name=service,proto3" json:"service,omitempty"` + // Type of operation to start. + Operation string `protobuf:"bytes,2,opt,name=operation,proto3" json:"operation,omitempty"` + // A request ID that can be used as an idempotentency key. + RequestId string `protobuf:"bytes,3,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` + // Callback URL to call upon completion if the started operation is async. + Callback string `protobuf:"bytes,4,opt,name=callback,proto3" json:"callback,omitempty"` + // Full request body from the incoming HTTP request. + Payload *v1.Payload `protobuf:"bytes,5,opt,name=payload,proto3" json:"payload,omitempty"` + // Header that is expected to be attached to the callback request when the operation completes. + CallbackHeader map[string]string `protobuf:"bytes,6,rep,name=callback_header,json=callbackHeader,proto3" json:"callback_header,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // Links contain caller information and can be attached to the operations started by the handler. + Links []*Link `protobuf:"bytes,7,rep,name=links,proto3" json:"links,omitempty"` +} + +func (x *StartOperationRequest) Reset() { + *x = StartOperationRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartOperationRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartOperationRequest) ProtoMessage() {} + +func (x *StartOperationRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartOperationRequest.ProtoReflect.Descriptor instead. +func (*StartOperationRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *StartOperationRequest) GetService() string { + if x != nil { + return x.Service + } + return "" +} + +func (x *StartOperationRequest) GetOperation() string { + if x != nil { + return x.Operation + } + return "" +} + +func (x *StartOperationRequest) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +func (x *StartOperationRequest) GetCallback() string { + if x != nil { + return x.Callback + } + return "" +} + +func (x *StartOperationRequest) GetPayload() *v1.Payload { + if x != nil { + return x.Payload + } + return nil +} + +func (x *StartOperationRequest) GetCallbackHeader() map[string]string { + if x != nil { + return x.CallbackHeader + } + return nil +} + +func (x *StartOperationRequest) GetLinks() []*Link { + if x != nil { + return x.Links + } + return nil +} + +// A request to cancel an operation. +type CancelOperationRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Service name. + Service string `protobuf:"bytes,1,opt,name=service,proto3" json:"service,omitempty"` + // Type of operation to cancel. + Operation string `protobuf:"bytes,2,opt,name=operation,proto3" json:"operation,omitempty"` + // Operation ID as originally generated by a Handler. + OperationId string `protobuf:"bytes,3,opt,name=operation_id,json=operationId,proto3" json:"operation_id,omitempty"` +} + +func (x *CancelOperationRequest) Reset() { + *x = CancelOperationRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CancelOperationRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CancelOperationRequest) ProtoMessage() {} + +func (x *CancelOperationRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CancelOperationRequest.ProtoReflect.Descriptor instead. +func (*CancelOperationRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{5} +} + +func (x *CancelOperationRequest) GetService() string { + if x != nil { + return x.Service + } + return "" +} + +func (x *CancelOperationRequest) GetOperation() string { + if x != nil { + return x.Operation + } + return "" +} + +func (x *CancelOperationRequest) GetOperationId() string { + if x != nil { + return x.OperationId + } + return "" +} + +// A Nexus request. +type Request struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Headers extracted from the original request in the Temporal frontend. + // When using Nexus over HTTP, this includes the request's HTTP headers ignoring multiple values. + Header map[string]string `protobuf:"bytes,1,rep,name=header,proto3" json:"header,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // The timestamp when the request was scheduled in the frontend. + // (-- api-linter: core::0142::time-field-names=disabled + // + // aip.dev/not-precedent: Not following linter rules. --) + ScheduledTime *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=scheduled_time,json=scheduledTime,proto3" json:"scheduled_time,omitempty"` + // Types that are assignable to Variant: + // + // *Request_StartOperation + // *Request_CancelOperation + Variant isRequest_Variant `protobuf_oneof:"variant"` +} + +func (x *Request) Reset() { + *x = Request{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Request) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Request) ProtoMessage() {} + +func (x *Request) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Request.ProtoReflect.Descriptor instead. +func (*Request) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{6} +} + +func (x *Request) GetHeader() map[string]string { + if x != nil { + return x.Header + } + return nil +} + +func (x *Request) GetScheduledTime() *timestamppb.Timestamp { + if x != nil { + return x.ScheduledTime + } + return nil +} + +func (m *Request) GetVariant() isRequest_Variant { + if m != nil { + return m.Variant + } + return nil +} + +func (x *Request) GetStartOperation() *StartOperationRequest { + if x, ok := x.GetVariant().(*Request_StartOperation); ok { + return x.StartOperation + } + return nil +} + +func (x *Request) GetCancelOperation() *CancelOperationRequest { + if x, ok := x.GetVariant().(*Request_CancelOperation); ok { + return x.CancelOperation + } + return nil +} + +type isRequest_Variant interface { + isRequest_Variant() +} + +type Request_StartOperation struct { + StartOperation *StartOperationRequest `protobuf:"bytes,3,opt,name=start_operation,json=startOperation,proto3,oneof"` +} + +type Request_CancelOperation struct { + CancelOperation *CancelOperationRequest `protobuf:"bytes,4,opt,name=cancel_operation,json=cancelOperation,proto3,oneof"` +} + +func (*Request_StartOperation) isRequest_Variant() {} + +func (*Request_CancelOperation) isRequest_Variant() {} + +// Response variant for StartOperationRequest. +type StartOperationResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Variant: + // + // *StartOperationResponse_SyncSuccess + // *StartOperationResponse_AsyncSuccess + // *StartOperationResponse_OperationError + Variant isStartOperationResponse_Variant `protobuf_oneof:"variant"` +} + +func (x *StartOperationResponse) Reset() { + *x = StartOperationResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartOperationResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartOperationResponse) ProtoMessage() {} + +func (x *StartOperationResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartOperationResponse.ProtoReflect.Descriptor instead. +func (*StartOperationResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{7} +} + +func (m *StartOperationResponse) GetVariant() isStartOperationResponse_Variant { + if m != nil { + return m.Variant + } + return nil +} + +func (x *StartOperationResponse) GetSyncSuccess() *StartOperationResponse_Sync { + if x, ok := x.GetVariant().(*StartOperationResponse_SyncSuccess); ok { + return x.SyncSuccess + } + return nil +} + +func (x *StartOperationResponse) GetAsyncSuccess() *StartOperationResponse_Async { + if x, ok := x.GetVariant().(*StartOperationResponse_AsyncSuccess); ok { + return x.AsyncSuccess + } + return nil +} + +func (x *StartOperationResponse) GetOperationError() *UnsuccessfulOperationError { + if x, ok := x.GetVariant().(*StartOperationResponse_OperationError); ok { + return x.OperationError + } + return nil +} + +type isStartOperationResponse_Variant interface { + isStartOperationResponse_Variant() +} + +type StartOperationResponse_SyncSuccess struct { + SyncSuccess *StartOperationResponse_Sync `protobuf:"bytes,1,opt,name=sync_success,json=syncSuccess,proto3,oneof"` +} + +type StartOperationResponse_AsyncSuccess struct { + AsyncSuccess *StartOperationResponse_Async `protobuf:"bytes,2,opt,name=async_success,json=asyncSuccess,proto3,oneof"` +} + +type StartOperationResponse_OperationError struct { + // The operation completed unsuccessfully (failed or canceled). + OperationError *UnsuccessfulOperationError `protobuf:"bytes,3,opt,name=operation_error,json=operationError,proto3,oneof"` +} + +func (*StartOperationResponse_SyncSuccess) isStartOperationResponse_Variant() {} + +func (*StartOperationResponse_AsyncSuccess) isStartOperationResponse_Variant() {} + +func (*StartOperationResponse_OperationError) isStartOperationResponse_Variant() {} + +// Response variant for CancelOperationRequest. +type CancelOperationResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CancelOperationResponse) Reset() { + *x = CancelOperationResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CancelOperationResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CancelOperationResponse) ProtoMessage() {} + +func (x *CancelOperationResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CancelOperationResponse.ProtoReflect.Descriptor instead. +func (*CancelOperationResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{8} +} + +// A response indicating that the handler has successfully processed a request. +type Response struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Variant must correlate to the corresponding Request's variant. + // + // Types that are assignable to Variant: + // + // *Response_StartOperation + // *Response_CancelOperation + Variant isResponse_Variant `protobuf_oneof:"variant"` +} + +func (x *Response) Reset() { + *x = Response{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Response) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Response) ProtoMessage() {} + +func (x *Response) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Response.ProtoReflect.Descriptor instead. +func (*Response) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{9} +} + +func (m *Response) GetVariant() isResponse_Variant { + if m != nil { + return m.Variant + } + return nil +} + +func (x *Response) GetStartOperation() *StartOperationResponse { + if x, ok := x.GetVariant().(*Response_StartOperation); ok { + return x.StartOperation + } + return nil +} + +func (x *Response) GetCancelOperation() *CancelOperationResponse { + if x, ok := x.GetVariant().(*Response_CancelOperation); ok { + return x.CancelOperation + } + return nil +} + +type isResponse_Variant interface { + isResponse_Variant() +} + +type Response_StartOperation struct { + StartOperation *StartOperationResponse `protobuf:"bytes,1,opt,name=start_operation,json=startOperation,proto3,oneof"` +} + +type Response_CancelOperation struct { + CancelOperation *CancelOperationResponse `protobuf:"bytes,2,opt,name=cancel_operation,json=cancelOperation,proto3,oneof"` +} + +func (*Response_StartOperation) isResponse_Variant() {} + +func (*Response_CancelOperation) isResponse_Variant() {} + +// A cluster-global binding from an endpoint ID to a target for dispatching incoming Nexus requests. +type Endpoint struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Data version for this endpoint, incremented for every update issued via the UpdateNexusEndpoint API. + Version int64 `protobuf:"varint,1,opt,name=version,proto3" json:"version,omitempty"` + // Unique server-generated endpoint ID. + Id string `protobuf:"bytes,2,opt,name=id,proto3" json:"id,omitempty"` + // Spec for the endpoint. + Spec *EndpointSpec `protobuf:"bytes,3,opt,name=spec,proto3" json:"spec,omitempty"` + // The date and time when the endpoint was created. + // (-- api-linter: core::0142::time-field-names=disabled + // + // aip.dev/not-precedent: Not following linter rules. --) + CreatedTime *timestamppb.Timestamp `protobuf:"bytes,4,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"` + // The date and time when the endpoint was last modified. + // Will not be set if the endpoint has never been modified. + // (-- api-linter: core::0142::time-field-names=disabled + // + // aip.dev/not-precedent: Not following linter rules. --) + LastModifiedTime *timestamppb.Timestamp `protobuf:"bytes,5,opt,name=last_modified_time,json=lastModifiedTime,proto3" json:"last_modified_time,omitempty"` + // Server exposed URL prefix for invocation of operations on this endpoint. + // This doesn't include the protocol, hostname or port as the server does not know how it should be accessed + // publicly. The URL is stable in the face of endpoint renames. + UrlPrefix string `protobuf:"bytes,6,opt,name=url_prefix,json=urlPrefix,proto3" json:"url_prefix,omitempty"` +} + +func (x *Endpoint) Reset() { + *x = Endpoint{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Endpoint) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Endpoint) ProtoMessage() {} + +func (x *Endpoint) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Endpoint.ProtoReflect.Descriptor instead. +func (*Endpoint) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{10} +} + +func (x *Endpoint) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *Endpoint) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *Endpoint) GetSpec() *EndpointSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *Endpoint) GetCreatedTime() *timestamppb.Timestamp { + if x != nil { + return x.CreatedTime + } + return nil +} + +func (x *Endpoint) GetLastModifiedTime() *timestamppb.Timestamp { + if x != nil { + return x.LastModifiedTime + } + return nil +} + +func (x *Endpoint) GetUrlPrefix() string { + if x != nil { + return x.UrlPrefix + } + return "" +} + +// Contains mutable fields for an Endpoint. +type EndpointSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Endpoint name, unique for this cluster. Must match `[a-zA-Z_][a-zA-Z0-9_]*`. + // Renaming an endpoint breaks all workflow callers that reference this endpoint, causing operations to fail. + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // Markdown description serialized as a single JSON string. + // If the Payload is encrypted, the UI and CLI may decrypt with the configured codec server endpoint. + // By default, the server enforces a limit of 20,000 bytes for this entire payload. + Description *v1.Payload `protobuf:"bytes,2,opt,name=description,proto3" json:"description,omitempty"` + // Target to route requests to. + Target *EndpointTarget `protobuf:"bytes,3,opt,name=target,proto3" json:"target,omitempty"` +} + +func (x *EndpointSpec) Reset() { + *x = EndpointSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *EndpointSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EndpointSpec) ProtoMessage() {} + +func (x *EndpointSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EndpointSpec.ProtoReflect.Descriptor instead. +func (*EndpointSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{11} +} + +func (x *EndpointSpec) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *EndpointSpec) GetDescription() *v1.Payload { + if x != nil { + return x.Description + } + return nil +} + +func (x *EndpointSpec) GetTarget() *EndpointTarget { + if x != nil { + return x.Target + } + return nil +} + +// Target to route requests to. +type EndpointTarget struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Variant: + // + // *EndpointTarget_Worker_ + // *EndpointTarget_External_ + Variant isEndpointTarget_Variant `protobuf_oneof:"variant"` +} + +func (x *EndpointTarget) Reset() { + *x = EndpointTarget{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *EndpointTarget) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EndpointTarget) ProtoMessage() {} + +func (x *EndpointTarget) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EndpointTarget.ProtoReflect.Descriptor instead. +func (*EndpointTarget) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{12} +} + +func (m *EndpointTarget) GetVariant() isEndpointTarget_Variant { + if m != nil { + return m.Variant + } + return nil +} + +func (x *EndpointTarget) GetWorker() *EndpointTarget_Worker { + if x, ok := x.GetVariant().(*EndpointTarget_Worker_); ok { + return x.Worker + } + return nil +} + +func (x *EndpointTarget) GetExternal() *EndpointTarget_External { + if x, ok := x.GetVariant().(*EndpointTarget_External_); ok { + return x.External + } + return nil +} + +type isEndpointTarget_Variant interface { + isEndpointTarget_Variant() +} + +type EndpointTarget_Worker_ struct { + Worker *EndpointTarget_Worker `protobuf:"bytes,1,opt,name=worker,proto3,oneof"` +} + +type EndpointTarget_External_ struct { + External *EndpointTarget_External `protobuf:"bytes,2,opt,name=external,proto3,oneof"` +} + +func (*EndpointTarget_Worker_) isEndpointTarget_Variant() {} + +func (*EndpointTarget_External_) isEndpointTarget_Variant() {} + +// An operation completed successfully. +type StartOperationResponse_Sync struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Payload *v1.Payload `protobuf:"bytes,1,opt,name=payload,proto3" json:"payload,omitempty"` +} + +func (x *StartOperationResponse_Sync) Reset() { + *x = StartOperationResponse_Sync{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartOperationResponse_Sync) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartOperationResponse_Sync) ProtoMessage() {} + +func (x *StartOperationResponse_Sync) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[16] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartOperationResponse_Sync.ProtoReflect.Descriptor instead. +func (*StartOperationResponse_Sync) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{7, 0} +} + +func (x *StartOperationResponse_Sync) GetPayload() *v1.Payload { + if x != nil { + return x.Payload + } + return nil +} + +// The operation will complete asynchronously. +// The returned ID can be used to reference this operation. +type StartOperationResponse_Async struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + OperationId string `protobuf:"bytes,1,opt,name=operation_id,json=operationId,proto3" json:"operation_id,omitempty"` + Links []*Link `protobuf:"bytes,2,rep,name=links,proto3" json:"links,omitempty"` +} + +func (x *StartOperationResponse_Async) Reset() { + *x = StartOperationResponse_Async{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartOperationResponse_Async) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartOperationResponse_Async) ProtoMessage() {} + +func (x *StartOperationResponse_Async) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[17] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartOperationResponse_Async.ProtoReflect.Descriptor instead. +func (*StartOperationResponse_Async) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{7, 1} +} + +func (x *StartOperationResponse_Async) GetOperationId() string { + if x != nil { + return x.OperationId + } + return "" +} + +func (x *StartOperationResponse_Async) GetLinks() []*Link { + if x != nil { + return x.Links + } + return nil +} + +// Target a worker polling on a Nexus task queue in a specific namespace. +type EndpointTarget_Worker struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace to route requests to. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Nexus task queue to route requests to. + TaskQueue string `protobuf:"bytes,2,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` +} + +func (x *EndpointTarget_Worker) Reset() { + *x = EndpointTarget_Worker{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *EndpointTarget_Worker) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EndpointTarget_Worker) ProtoMessage() {} + +func (x *EndpointTarget_Worker) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[18] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EndpointTarget_Worker.ProtoReflect.Descriptor instead. +func (*EndpointTarget_Worker) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{12, 0} +} + +func (x *EndpointTarget_Worker) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *EndpointTarget_Worker) GetTaskQueue() string { + if x != nil { + return x.TaskQueue + } + return "" +} + +// Target an external server by URL. +// At a later point, this will support providing credentials, in the meantime, an http.RoundTripper can be injected +// into the server to modify the request. +type EndpointTarget_External struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // URL to call. + Url string `protobuf:"bytes,1,opt,name=url,proto3" json:"url,omitempty"` +} + +func (x *EndpointTarget_External) Reset() { + *x = EndpointTarget_External{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *EndpointTarget_External) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EndpointTarget_External) ProtoMessage() {} + +func (x *EndpointTarget_External) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_nexus_v1_message_proto_msgTypes[19] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EndpointTarget_External.ProtoReflect.Descriptor instead. +func (*EndpointTarget_External) Descriptor() ([]byte, []int) { + return file_temporal_api_nexus_v1_message_proto_rawDescGZIP(), []int{12, 1} +} + +func (x *EndpointTarget_External) GetUrl() string { + if x != nil { + return x.Url + } + return "" +} + +var File_temporal_api_nexus_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_nexus_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x23, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x6e, 0x65, 0x78, + 0x75, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xd8, 0x01, 0x0a, 0x07, 0x46, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x12, 0x1c, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x4c, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, + 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x64, + 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x64, 0x65, 0x74, + 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x43, 0x0a, 0x0d, 0x4d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x6f, 0x0a, 0x0c, 0x48, 0x61, 0x6e, 0x64, 0x6c, + 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x21, 0x0a, 0x0a, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, + 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3c, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x46, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x87, 0x01, 0x0a, 0x1a, 0x55, 0x6e, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x66, + 0x75, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, + 0x2b, 0x0a, 0x0f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3c, 0x0a, 0x07, 0x66, 0x61, 0x69, + 0x6c, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, + 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x22, 0x34, 0x0a, 0x04, 0x4c, 0x69, 0x6e, 0x6b, 0x12, 0x14, 0x0a, 0x03, 0x75, + 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x16, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, + 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xca, 0x03, 0x0a, 0x15, 0x53, 0x74, 0x61, 0x72, 0x74, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x1c, 0x0a, 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x63, + 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x61, + 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x07, 0x70, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x6d, 0x0a, 0x0f, 0x63, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, + 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, + 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, + 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x63, 0x61, 0x6c, 0x6c, + 0x62, 0x61, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x35, 0x0a, + 0x05, 0x6c, 0x69, 0x6e, 0x6b, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, + 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x6e, 0x6b, 0x52, 0x05, 0x6c, 0x69, 0x6e, 0x6b, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x1a, 0x49, 0x0a, 0x13, 0x43, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x7f, 0x0a, 0x16, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x1c, 0x0a, 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x22, 0xa3, 0x03, 0x0a, 0x07, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x06, 0x68, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, + 0x31, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x45, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x5b, 0x0a, 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x74, 0x61, 0x72, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5e, 0x0a, 0x10, 0x63, 0x61, 0x6e, 0x63, + 0x65, 0x6c, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, + 0x0f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x1a, 0x41, 0x0a, 0x0b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, + 0x38, 0x01, 0x42, 0x09, 0x0a, 0x07, 0x76, 0x61, 0x72, 0x69, 0x61, 0x6e, 0x74, 0x22, 0xf0, 0x03, 0x0a, + 0x16, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5b, 0x0a, 0x0c, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x73, 0x75, + 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, + 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x79, + 0x6e, 0x63, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5e, 0x0a, 0x0d, + 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, + 0x73, 0x79, 0x6e, 0x63, 0x48, 0x00, 0x52, 0x0c, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x53, 0x75, 0x63, 0x63, + 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x60, 0x0a, 0x0f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, + 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x6e, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x66, 0x75, + 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x48, 0x00, + 0x52, 0x0e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x42, + 0x02, 0x68, 0x00, 0x1a, 0x45, 0x0a, 0x04, 0x53, 0x79, 0x6e, 0x63, 0x12, 0x3d, 0x0a, 0x07, 0x70, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x65, 0x0a, 0x05, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x12, 0x25, + 0x0a, 0x0c, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x35, 0x0a, 0x05, 0x6c, 0x69, 0x6e, 0x6b, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x1b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, + 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x6e, 0x6b, 0x52, 0x05, 0x6c, 0x69, 0x6e, + 0x6b, 0x73, 0x42, 0x02, 0x68, 0x00, 0x42, 0x09, 0x0a, 0x07, 0x76, 0x61, 0x72, 0x69, 0x61, 0x6e, 0x74, + 0x22, 0x19, 0x0a, 0x17, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xd4, 0x01, 0x0a, 0x08, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, 0x0a, 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, + 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5f, + 0x0a, 0x10, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, + 0x6e, 0x63, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x0f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x42, 0x09, 0x0a, 0x07, 0x76, 0x61, 0x72, + 0x69, 0x61, 0x6e, 0x74, 0x22, 0xad, 0x02, 0x0a, 0x08, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, + 0x12, 0x1c, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x12, 0x0a, 0x02, + 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x3b, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, + 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, + 0x73, 0x70, 0x65, 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, + 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4c, 0x0a, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, + 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x10, 0x6c, 0x61, 0x73, + 0x74, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x21, 0x0a, 0x0a, 0x75, 0x72, 0x6c, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x75, 0x72, 0x6c, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x42, 0x02, 0x68, + 0x00, 0x22, 0xb0, 0x01, 0x0a, 0x0c, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x53, 0x70, 0x65, + 0x63, 0x12, 0x16, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x45, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x06, 0x74, 0x61, 0x72, + 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, + 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, + 0x61, 0x72, 0x67, 0x65, 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0xaa, 0x02, 0x0a, 0x0e, 0x45, 0x6e, 0x64, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x4a, 0x0a, 0x06, 0x77, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, + 0x2e, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x2e, 0x57, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x48, 0x00, 0x52, 0x06, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x50, 0x0a, 0x08, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x2e, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x48, 0x00, 0x52, 0x08, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x1a, + 0x4d, 0x0a, 0x06, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, + 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x73, + 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x20, 0x0a, 0x08, 0x45, 0x78, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x14, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x42, 0x09, 0x0a, 0x07, 0x76, 0x61, 0x72, 0x69, + 0x61, 0x6e, 0x74, 0x42, 0x84, 0x01, 0x0a, 0x18, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x42, 0x0c, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x21, 0x67, + 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, + 0x2f, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2f, 0x76, 0x31, 0x3b, 0x6e, 0x65, 0x78, 0x75, 0x73, 0xaa, 0x02, + 0x17, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x4e, + 0x65, 0x78, 0x75, 0x73, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1a, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x3a, 0x3a, + 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_nexus_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_nexus_v1_message_proto_rawDescData = file_temporal_api_nexus_v1_message_proto_rawDesc +) + +func file_temporal_api_nexus_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_nexus_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_nexus_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_nexus_v1_message_proto_rawDescData) + }) + return file_temporal_api_nexus_v1_message_proto_rawDescData +} + +var file_temporal_api_nexus_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 20) +var file_temporal_api_nexus_v1_message_proto_goTypes = []any{ + (*Failure)(nil), // 0: temporal.api.nexus.v1.Failure + (*HandlerError)(nil), // 1: temporal.api.nexus.v1.HandlerError + (*UnsuccessfulOperationError)(nil), // 2: temporal.api.nexus.v1.UnsuccessfulOperationError + (*Link)(nil), // 3: temporal.api.nexus.v1.Link + (*StartOperationRequest)(nil), // 4: temporal.api.nexus.v1.StartOperationRequest + (*CancelOperationRequest)(nil), // 5: temporal.api.nexus.v1.CancelOperationRequest + (*Request)(nil), // 6: temporal.api.nexus.v1.Request + (*StartOperationResponse)(nil), // 7: temporal.api.nexus.v1.StartOperationResponse + (*CancelOperationResponse)(nil), // 8: temporal.api.nexus.v1.CancelOperationResponse + (*Response)(nil), // 9: temporal.api.nexus.v1.Response + (*Endpoint)(nil), // 10: temporal.api.nexus.v1.Endpoint + (*EndpointSpec)(nil), // 11: temporal.api.nexus.v1.EndpointSpec + (*EndpointTarget)(nil), // 12: temporal.api.nexus.v1.EndpointTarget + nil, // 13: temporal.api.nexus.v1.Failure.MetadataEntry + nil, // 14: temporal.api.nexus.v1.StartOperationRequest.CallbackHeaderEntry + nil, // 15: temporal.api.nexus.v1.Request.HeaderEntry + (*StartOperationResponse_Sync)(nil), // 16: temporal.api.nexus.v1.StartOperationResponse.Sync + (*StartOperationResponse_Async)(nil), // 17: temporal.api.nexus.v1.StartOperationResponse.Async + (*EndpointTarget_Worker)(nil), // 18: temporal.api.nexus.v1.EndpointTarget.Worker + (*EndpointTarget_External)(nil), // 19: temporal.api.nexus.v1.EndpointTarget.External + (*v1.Payload)(nil), // 20: temporal.api.common.v1.Payload + (*timestamppb.Timestamp)(nil), // 21: google.protobuf.Timestamp +} +var file_temporal_api_nexus_v1_message_proto_depIdxs = []int32{ + 13, // 0: temporal.api.nexus.v1.Failure.metadata:type_name -> temporal.api.nexus.v1.Failure.MetadataEntry + 0, // 1: temporal.api.nexus.v1.HandlerError.failure:type_name -> temporal.api.nexus.v1.Failure + 0, // 2: temporal.api.nexus.v1.UnsuccessfulOperationError.failure:type_name -> temporal.api.nexus.v1.Failure + 20, // 3: temporal.api.nexus.v1.StartOperationRequest.payload:type_name -> temporal.api.common.v1.Payload + 14, // 4: temporal.api.nexus.v1.StartOperationRequest.callback_header:type_name -> temporal.api.nexus.v1.StartOperationRequest.CallbackHeaderEntry + 3, // 5: temporal.api.nexus.v1.StartOperationRequest.links:type_name -> temporal.api.nexus.v1.Link + 15, // 6: temporal.api.nexus.v1.Request.header:type_name -> temporal.api.nexus.v1.Request.HeaderEntry + 21, // 7: temporal.api.nexus.v1.Request.scheduled_time:type_name -> google.protobuf.Timestamp + 4, // 8: temporal.api.nexus.v1.Request.start_operation:type_name -> temporal.api.nexus.v1.StartOperationRequest + 5, // 9: temporal.api.nexus.v1.Request.cancel_operation:type_name -> temporal.api.nexus.v1.CancelOperationRequest + 16, // 10: temporal.api.nexus.v1.StartOperationResponse.sync_success:type_name -> temporal.api.nexus.v1.StartOperationResponse.Sync + 17, // 11: temporal.api.nexus.v1.StartOperationResponse.async_success:type_name -> temporal.api.nexus.v1.StartOperationResponse.Async + 2, // 12: temporal.api.nexus.v1.StartOperationResponse.operation_error:type_name -> temporal.api.nexus.v1.UnsuccessfulOperationError + 7, // 13: temporal.api.nexus.v1.Response.start_operation:type_name -> temporal.api.nexus.v1.StartOperationResponse + 8, // 14: temporal.api.nexus.v1.Response.cancel_operation:type_name -> temporal.api.nexus.v1.CancelOperationResponse + 11, // 15: temporal.api.nexus.v1.Endpoint.spec:type_name -> temporal.api.nexus.v1.EndpointSpec + 21, // 16: temporal.api.nexus.v1.Endpoint.created_time:type_name -> google.protobuf.Timestamp + 21, // 17: temporal.api.nexus.v1.Endpoint.last_modified_time:type_name -> google.protobuf.Timestamp + 20, // 18: temporal.api.nexus.v1.EndpointSpec.description:type_name -> temporal.api.common.v1.Payload + 12, // 19: temporal.api.nexus.v1.EndpointSpec.target:type_name -> temporal.api.nexus.v1.EndpointTarget + 18, // 20: temporal.api.nexus.v1.EndpointTarget.worker:type_name -> temporal.api.nexus.v1.EndpointTarget.Worker + 19, // 21: temporal.api.nexus.v1.EndpointTarget.external:type_name -> temporal.api.nexus.v1.EndpointTarget.External + 20, // 22: temporal.api.nexus.v1.StartOperationResponse.Sync.payload:type_name -> temporal.api.common.v1.Payload + 3, // 23: temporal.api.nexus.v1.StartOperationResponse.Async.links:type_name -> temporal.api.nexus.v1.Link + 24, // [24:24] is the sub-list for method output_type + 24, // [24:24] is the sub-list for method input_type + 24, // [24:24] is the sub-list for extension type_name + 24, // [24:24] is the sub-list for extension extendee + 0, // [0:24] is the sub-list for field type_name +} + +func init() { file_temporal_api_nexus_v1_message_proto_init() } +func file_temporal_api_nexus_v1_message_proto_init() { + if File_temporal_api_nexus_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_nexus_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*Failure); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*HandlerError); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*UnsuccessfulOperationError); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*Link); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*StartOperationRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*CancelOperationRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*Request); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*StartOperationResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*CancelOperationResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*Response); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*Endpoint); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[11].Exporter = func(v any, i int) any { + switch v := v.(*EndpointSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[12].Exporter = func(v any, i int) any { + switch v := v.(*EndpointTarget); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[16].Exporter = func(v any, i int) any { + switch v := v.(*StartOperationResponse_Sync); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[17].Exporter = func(v any, i int) any { + switch v := v.(*StartOperationResponse_Async); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[18].Exporter = func(v any, i int) any { + switch v := v.(*EndpointTarget_Worker); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[19].Exporter = func(v any, i int) any { + switch v := v.(*EndpointTarget_External); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_temporal_api_nexus_v1_message_proto_msgTypes[6].OneofWrappers = []any{ + (*Request_StartOperation)(nil), + (*Request_CancelOperation)(nil), + } + file_temporal_api_nexus_v1_message_proto_msgTypes[7].OneofWrappers = []any{ + (*StartOperationResponse_SyncSuccess)(nil), + (*StartOperationResponse_AsyncSuccess)(nil), + (*StartOperationResponse_OperationError)(nil), + } + file_temporal_api_nexus_v1_message_proto_msgTypes[9].OneofWrappers = []any{ + (*Response_StartOperation)(nil), + (*Response_CancelOperation)(nil), + } + file_temporal_api_nexus_v1_message_proto_msgTypes[12].OneofWrappers = []any{ + (*EndpointTarget_Worker_)(nil), + (*EndpointTarget_External_)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_nexus_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 20, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_nexus_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_nexus_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_nexus_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_nexus_v1_message_proto = out.File + file_temporal_api_nexus_v1_message_proto_rawDesc = nil + file_temporal_api_nexus_v1_message_proto_goTypes = nil + file_temporal_api_nexus_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/operatorservice/v1/request_response.go-helpers.pb.go b/vendor/go.temporal.io/api/operatorservice/v1/request_response.go-helpers.pb.go new file mode 100644 index 00000000000..7ea99ce7cfc --- /dev/null +++ b/vendor/go.temporal.io/api/operatorservice/v1/request_response.go-helpers.pb.go @@ -0,0 +1,953 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package operatorservice + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type AddSearchAttributesRequest to the protobuf v3 wire format +func (val *AddSearchAttributesRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type AddSearchAttributesRequest from the protobuf v3 wire format +func (val *AddSearchAttributesRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *AddSearchAttributesRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two AddSearchAttributesRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *AddSearchAttributesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *AddSearchAttributesRequest + switch t := that.(type) { + case *AddSearchAttributesRequest: + that1 = t + case AddSearchAttributesRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type AddSearchAttributesResponse to the protobuf v3 wire format +func (val *AddSearchAttributesResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type AddSearchAttributesResponse from the protobuf v3 wire format +func (val *AddSearchAttributesResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *AddSearchAttributesResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two AddSearchAttributesResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *AddSearchAttributesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *AddSearchAttributesResponse + switch t := that.(type) { + case *AddSearchAttributesResponse: + that1 = t + case AddSearchAttributesResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RemoveSearchAttributesRequest to the protobuf v3 wire format +func (val *RemoveSearchAttributesRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RemoveSearchAttributesRequest from the protobuf v3 wire format +func (val *RemoveSearchAttributesRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RemoveSearchAttributesRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RemoveSearchAttributesRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RemoveSearchAttributesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RemoveSearchAttributesRequest + switch t := that.(type) { + case *RemoveSearchAttributesRequest: + that1 = t + case RemoveSearchAttributesRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RemoveSearchAttributesResponse to the protobuf v3 wire format +func (val *RemoveSearchAttributesResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RemoveSearchAttributesResponse from the protobuf v3 wire format +func (val *RemoveSearchAttributesResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RemoveSearchAttributesResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RemoveSearchAttributesResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RemoveSearchAttributesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RemoveSearchAttributesResponse + switch t := that.(type) { + case *RemoveSearchAttributesResponse: + that1 = t + case RemoveSearchAttributesResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListSearchAttributesRequest to the protobuf v3 wire format +func (val *ListSearchAttributesRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListSearchAttributesRequest from the protobuf v3 wire format +func (val *ListSearchAttributesRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListSearchAttributesRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListSearchAttributesRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListSearchAttributesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListSearchAttributesRequest + switch t := that.(type) { + case *ListSearchAttributesRequest: + that1 = t + case ListSearchAttributesRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListSearchAttributesResponse to the protobuf v3 wire format +func (val *ListSearchAttributesResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListSearchAttributesResponse from the protobuf v3 wire format +func (val *ListSearchAttributesResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListSearchAttributesResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListSearchAttributesResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListSearchAttributesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListSearchAttributesResponse + switch t := that.(type) { + case *ListSearchAttributesResponse: + that1 = t + case ListSearchAttributesResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteNamespaceRequest to the protobuf v3 wire format +func (val *DeleteNamespaceRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteNamespaceRequest from the protobuf v3 wire format +func (val *DeleteNamespaceRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteNamespaceRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteNamespaceRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteNamespaceRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteNamespaceRequest + switch t := that.(type) { + case *DeleteNamespaceRequest: + that1 = t + case DeleteNamespaceRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteNamespaceResponse to the protobuf v3 wire format +func (val *DeleteNamespaceResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteNamespaceResponse from the protobuf v3 wire format +func (val *DeleteNamespaceResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteNamespaceResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteNamespaceResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteNamespaceResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteNamespaceResponse + switch t := that.(type) { + case *DeleteNamespaceResponse: + that1 = t + case DeleteNamespaceResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type AddOrUpdateRemoteClusterRequest to the protobuf v3 wire format +func (val *AddOrUpdateRemoteClusterRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type AddOrUpdateRemoteClusterRequest from the protobuf v3 wire format +func (val *AddOrUpdateRemoteClusterRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *AddOrUpdateRemoteClusterRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two AddOrUpdateRemoteClusterRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *AddOrUpdateRemoteClusterRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *AddOrUpdateRemoteClusterRequest + switch t := that.(type) { + case *AddOrUpdateRemoteClusterRequest: + that1 = t + case AddOrUpdateRemoteClusterRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type AddOrUpdateRemoteClusterResponse to the protobuf v3 wire format +func (val *AddOrUpdateRemoteClusterResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type AddOrUpdateRemoteClusterResponse from the protobuf v3 wire format +func (val *AddOrUpdateRemoteClusterResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *AddOrUpdateRemoteClusterResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two AddOrUpdateRemoteClusterResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *AddOrUpdateRemoteClusterResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *AddOrUpdateRemoteClusterResponse + switch t := that.(type) { + case *AddOrUpdateRemoteClusterResponse: + that1 = t + case AddOrUpdateRemoteClusterResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RemoveRemoteClusterRequest to the protobuf v3 wire format +func (val *RemoveRemoteClusterRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RemoveRemoteClusterRequest from the protobuf v3 wire format +func (val *RemoveRemoteClusterRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RemoveRemoteClusterRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RemoveRemoteClusterRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RemoveRemoteClusterRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RemoveRemoteClusterRequest + switch t := that.(type) { + case *RemoveRemoteClusterRequest: + that1 = t + case RemoveRemoteClusterRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RemoveRemoteClusterResponse to the protobuf v3 wire format +func (val *RemoveRemoteClusterResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RemoveRemoteClusterResponse from the protobuf v3 wire format +func (val *RemoveRemoteClusterResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RemoveRemoteClusterResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RemoveRemoteClusterResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RemoveRemoteClusterResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RemoveRemoteClusterResponse + switch t := that.(type) { + case *RemoveRemoteClusterResponse: + that1 = t + case RemoveRemoteClusterResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListClustersRequest to the protobuf v3 wire format +func (val *ListClustersRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListClustersRequest from the protobuf v3 wire format +func (val *ListClustersRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListClustersRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListClustersRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListClustersRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListClustersRequest + switch t := that.(type) { + case *ListClustersRequest: + that1 = t + case ListClustersRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListClustersResponse to the protobuf v3 wire format +func (val *ListClustersResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListClustersResponse from the protobuf v3 wire format +func (val *ListClustersResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListClustersResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListClustersResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListClustersResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListClustersResponse + switch t := that.(type) { + case *ListClustersResponse: + that1 = t + case ListClustersResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ClusterMetadata to the protobuf v3 wire format +func (val *ClusterMetadata) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ClusterMetadata from the protobuf v3 wire format +func (val *ClusterMetadata) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ClusterMetadata) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ClusterMetadata values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ClusterMetadata) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ClusterMetadata + switch t := that.(type) { + case *ClusterMetadata: + that1 = t + case ClusterMetadata: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetNexusEndpointRequest to the protobuf v3 wire format +func (val *GetNexusEndpointRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetNexusEndpointRequest from the protobuf v3 wire format +func (val *GetNexusEndpointRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetNexusEndpointRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetNexusEndpointRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetNexusEndpointRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetNexusEndpointRequest + switch t := that.(type) { + case *GetNexusEndpointRequest: + that1 = t + case GetNexusEndpointRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetNexusEndpointResponse to the protobuf v3 wire format +func (val *GetNexusEndpointResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetNexusEndpointResponse from the protobuf v3 wire format +func (val *GetNexusEndpointResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetNexusEndpointResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetNexusEndpointResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetNexusEndpointResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetNexusEndpointResponse + switch t := that.(type) { + case *GetNexusEndpointResponse: + that1 = t + case GetNexusEndpointResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateNexusEndpointRequest to the protobuf v3 wire format +func (val *CreateNexusEndpointRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateNexusEndpointRequest from the protobuf v3 wire format +func (val *CreateNexusEndpointRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateNexusEndpointRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateNexusEndpointRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateNexusEndpointRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateNexusEndpointRequest + switch t := that.(type) { + case *CreateNexusEndpointRequest: + that1 = t + case CreateNexusEndpointRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateNexusEndpointResponse to the protobuf v3 wire format +func (val *CreateNexusEndpointResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateNexusEndpointResponse from the protobuf v3 wire format +func (val *CreateNexusEndpointResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateNexusEndpointResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateNexusEndpointResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateNexusEndpointResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateNexusEndpointResponse + switch t := that.(type) { + case *CreateNexusEndpointResponse: + that1 = t + case CreateNexusEndpointResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateNexusEndpointRequest to the protobuf v3 wire format +func (val *UpdateNexusEndpointRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateNexusEndpointRequest from the protobuf v3 wire format +func (val *UpdateNexusEndpointRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateNexusEndpointRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateNexusEndpointRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateNexusEndpointRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateNexusEndpointRequest + switch t := that.(type) { + case *UpdateNexusEndpointRequest: + that1 = t + case UpdateNexusEndpointRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateNexusEndpointResponse to the protobuf v3 wire format +func (val *UpdateNexusEndpointResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateNexusEndpointResponse from the protobuf v3 wire format +func (val *UpdateNexusEndpointResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateNexusEndpointResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateNexusEndpointResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateNexusEndpointResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateNexusEndpointResponse + switch t := that.(type) { + case *UpdateNexusEndpointResponse: + that1 = t + case UpdateNexusEndpointResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteNexusEndpointRequest to the protobuf v3 wire format +func (val *DeleteNexusEndpointRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteNexusEndpointRequest from the protobuf v3 wire format +func (val *DeleteNexusEndpointRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteNexusEndpointRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteNexusEndpointRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteNexusEndpointRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteNexusEndpointRequest + switch t := that.(type) { + case *DeleteNexusEndpointRequest: + that1 = t + case DeleteNexusEndpointRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteNexusEndpointResponse to the protobuf v3 wire format +func (val *DeleteNexusEndpointResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteNexusEndpointResponse from the protobuf v3 wire format +func (val *DeleteNexusEndpointResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteNexusEndpointResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteNexusEndpointResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteNexusEndpointResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteNexusEndpointResponse + switch t := that.(type) { + case *DeleteNexusEndpointResponse: + that1 = t + case DeleteNexusEndpointResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListNexusEndpointsRequest to the protobuf v3 wire format +func (val *ListNexusEndpointsRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListNexusEndpointsRequest from the protobuf v3 wire format +func (val *ListNexusEndpointsRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListNexusEndpointsRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListNexusEndpointsRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListNexusEndpointsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListNexusEndpointsRequest + switch t := that.(type) { + case *ListNexusEndpointsRequest: + that1 = t + case ListNexusEndpointsRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListNexusEndpointsResponse to the protobuf v3 wire format +func (val *ListNexusEndpointsResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListNexusEndpointsResponse from the protobuf v3 wire format +func (val *ListNexusEndpointsResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListNexusEndpointsResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListNexusEndpointsResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListNexusEndpointsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListNexusEndpointsResponse + switch t := that.(type) { + case *ListNexusEndpointsResponse: + that1 = t + case ListNexusEndpointsResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/operatorservice/v1/request_response.pb.go b/vendor/go.temporal.io/api/operatorservice/v1/request_response.pb.go new file mode 100644 index 00000000000..90dbe359c6e --- /dev/null +++ b/vendor/go.temporal.io/api/operatorservice/v1/request_response.pb.go @@ -0,0 +1,2023 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/operatorservice/v1/request_response.proto + +package operatorservice + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/enums/v1" + v11 "go.temporal.io/api/nexus/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + durationpb "google.golang.org/protobuf/types/known/durationpb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type AddSearchAttributesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Mapping between search attribute name and its IndexedValueType. + SearchAttributes map[string]v1.IndexedValueType `protobuf:"bytes,1,rep,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3,enum=temporal.api.enums.v1.IndexedValueType"` + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` +} + +func (x *AddSearchAttributesRequest) Reset() { + *x = AddSearchAttributesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AddSearchAttributesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AddSearchAttributesRequest) ProtoMessage() {} + +func (x *AddSearchAttributesRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AddSearchAttributesRequest.ProtoReflect.Descriptor instead. +func (*AddSearchAttributesRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{0} +} + +func (x *AddSearchAttributesRequest) GetSearchAttributes() map[string]v1.IndexedValueType { + if x != nil { + return x.SearchAttributes + } + return nil +} + +func (x *AddSearchAttributesRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +type AddSearchAttributesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *AddSearchAttributesResponse) Reset() { + *x = AddSearchAttributesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AddSearchAttributesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AddSearchAttributesResponse) ProtoMessage() {} + +func (x *AddSearchAttributesResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AddSearchAttributesResponse.ProtoReflect.Descriptor instead. +func (*AddSearchAttributesResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{1} +} + +type RemoveSearchAttributesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Search attribute names to delete. + SearchAttributes []string `protobuf:"bytes,1,rep,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` +} + +func (x *RemoveSearchAttributesRequest) Reset() { + *x = RemoveSearchAttributesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RemoveSearchAttributesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RemoveSearchAttributesRequest) ProtoMessage() {} + +func (x *RemoveSearchAttributesRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RemoveSearchAttributesRequest.ProtoReflect.Descriptor instead. +func (*RemoveSearchAttributesRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{2} +} + +func (x *RemoveSearchAttributesRequest) GetSearchAttributes() []string { + if x != nil { + return x.SearchAttributes + } + return nil +} + +func (x *RemoveSearchAttributesRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +type RemoveSearchAttributesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RemoveSearchAttributesResponse) Reset() { + *x = RemoveSearchAttributesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RemoveSearchAttributesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RemoveSearchAttributesResponse) ProtoMessage() {} + +func (x *RemoveSearchAttributesResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RemoveSearchAttributesResponse.ProtoReflect.Descriptor instead. +func (*RemoveSearchAttributesResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{3} +} + +type ListSearchAttributesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` +} + +func (x *ListSearchAttributesRequest) Reset() { + *x = ListSearchAttributesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListSearchAttributesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListSearchAttributesRequest) ProtoMessage() {} + +func (x *ListSearchAttributesRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListSearchAttributesRequest.ProtoReflect.Descriptor instead. +func (*ListSearchAttributesRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{4} +} + +func (x *ListSearchAttributesRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +type ListSearchAttributesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Mapping between custom (user-registered) search attribute name to its IndexedValueType. + CustomAttributes map[string]v1.IndexedValueType `protobuf:"bytes,1,rep,name=custom_attributes,json=customAttributes,proto3" json:"custom_attributes,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3,enum=temporal.api.enums.v1.IndexedValueType"` + // Mapping between system (predefined) search attribute name to its IndexedValueType. + SystemAttributes map[string]v1.IndexedValueType `protobuf:"bytes,2,rep,name=system_attributes,json=systemAttributes,proto3" json:"system_attributes,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3,enum=temporal.api.enums.v1.IndexedValueType"` + // Mapping from the attribute name to the visibility storage native type. + StorageSchema map[string]string `protobuf:"bytes,3,rep,name=storage_schema,json=storageSchema,proto3" json:"storage_schema,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *ListSearchAttributesResponse) Reset() { + *x = ListSearchAttributesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListSearchAttributesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListSearchAttributesResponse) ProtoMessage() {} + +func (x *ListSearchAttributesResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListSearchAttributesResponse.ProtoReflect.Descriptor instead. +func (*ListSearchAttributesResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{5} +} + +func (x *ListSearchAttributesResponse) GetCustomAttributes() map[string]v1.IndexedValueType { + if x != nil { + return x.CustomAttributes + } + return nil +} + +func (x *ListSearchAttributesResponse) GetSystemAttributes() map[string]v1.IndexedValueType { + if x != nil { + return x.SystemAttributes + } + return nil +} + +func (x *ListSearchAttributesResponse) GetStorageSchema() map[string]string { + if x != nil { + return x.StorageSchema + } + return nil +} + +type DeleteNamespaceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Only one of namespace or namespace_id must be specified to identify namespace. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + NamespaceId string `protobuf:"bytes,2,opt,name=namespace_id,json=namespaceId,proto3" json:"namespace_id,omitempty"` + // If provided, the deletion of namespace info will be delayed for the given duration (0 means no delay). + // If not provided, the default delay configured in the cluster will be used. + NamespaceDeleteDelay *durationpb.Duration `protobuf:"bytes,3,opt,name=namespace_delete_delay,json=namespaceDeleteDelay,proto3" json:"namespace_delete_delay,omitempty"` +} + +func (x *DeleteNamespaceRequest) Reset() { + *x = DeleteNamespaceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteNamespaceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteNamespaceRequest) ProtoMessage() {} + +func (x *DeleteNamespaceRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteNamespaceRequest.ProtoReflect.Descriptor instead. +func (*DeleteNamespaceRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{6} +} + +func (x *DeleteNamespaceRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *DeleteNamespaceRequest) GetNamespaceId() string { + if x != nil { + return x.NamespaceId + } + return "" +} + +func (x *DeleteNamespaceRequest) GetNamespaceDeleteDelay() *durationpb.Duration { + if x != nil { + return x.NamespaceDeleteDelay + } + return nil +} + +type DeleteNamespaceResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Temporary namespace name that is used during reclaim resources step. + DeletedNamespace string `protobuf:"bytes,1,opt,name=deleted_namespace,json=deletedNamespace,proto3" json:"deleted_namespace,omitempty"` +} + +func (x *DeleteNamespaceResponse) Reset() { + *x = DeleteNamespaceResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteNamespaceResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteNamespaceResponse) ProtoMessage() {} + +func (x *DeleteNamespaceResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteNamespaceResponse.ProtoReflect.Descriptor instead. +func (*DeleteNamespaceResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{7} +} + +func (x *DeleteNamespaceResponse) GetDeletedNamespace() string { + if x != nil { + return x.DeletedNamespace + } + return "" +} + +type AddOrUpdateRemoteClusterRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Frontend Address is a cross cluster accessible address for gRPC traffic. This field is required. + FrontendAddress string `protobuf:"bytes,1,opt,name=frontend_address,json=frontendAddress,proto3" json:"frontend_address,omitempty"` + // Flag to enable / disable the cross cluster connection. + EnableRemoteClusterConnection bool `protobuf:"varint,2,opt,name=enable_remote_cluster_connection,json=enableRemoteClusterConnection,proto3" json:"enable_remote_cluster_connection,omitempty"` + // Frontend HTTP Address is a cross cluster accessible address for HTTP traffic. This field is optional. If not provided + // + // on update, the existing HTTP address will be removed. + FrontendHttpAddress string `protobuf:"bytes,3,opt,name=frontend_http_address,json=frontendHttpAddress,proto3" json:"frontend_http_address,omitempty"` +} + +func (x *AddOrUpdateRemoteClusterRequest) Reset() { + *x = AddOrUpdateRemoteClusterRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AddOrUpdateRemoteClusterRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AddOrUpdateRemoteClusterRequest) ProtoMessage() {} + +func (x *AddOrUpdateRemoteClusterRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AddOrUpdateRemoteClusterRequest.ProtoReflect.Descriptor instead. +func (*AddOrUpdateRemoteClusterRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{8} +} + +func (x *AddOrUpdateRemoteClusterRequest) GetFrontendAddress() string { + if x != nil { + return x.FrontendAddress + } + return "" +} + +func (x *AddOrUpdateRemoteClusterRequest) GetEnableRemoteClusterConnection() bool { + if x != nil { + return x.EnableRemoteClusterConnection + } + return false +} + +func (x *AddOrUpdateRemoteClusterRequest) GetFrontendHttpAddress() string { + if x != nil { + return x.FrontendHttpAddress + } + return "" +} + +type AddOrUpdateRemoteClusterResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *AddOrUpdateRemoteClusterResponse) Reset() { + *x = AddOrUpdateRemoteClusterResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AddOrUpdateRemoteClusterResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AddOrUpdateRemoteClusterResponse) ProtoMessage() {} + +func (x *AddOrUpdateRemoteClusterResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AddOrUpdateRemoteClusterResponse.ProtoReflect.Descriptor instead. +func (*AddOrUpdateRemoteClusterResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{9} +} + +type RemoveRemoteClusterRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Remote cluster name to be removed. + ClusterName string `protobuf:"bytes,1,opt,name=cluster_name,json=clusterName,proto3" json:"cluster_name,omitempty"` +} + +func (x *RemoveRemoteClusterRequest) Reset() { + *x = RemoveRemoteClusterRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RemoveRemoteClusterRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RemoveRemoteClusterRequest) ProtoMessage() {} + +func (x *RemoveRemoteClusterRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RemoveRemoteClusterRequest.ProtoReflect.Descriptor instead. +func (*RemoveRemoteClusterRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{10} +} + +func (x *RemoveRemoteClusterRequest) GetClusterName() string { + if x != nil { + return x.ClusterName + } + return "" +} + +type RemoveRemoteClusterResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RemoveRemoteClusterResponse) Reset() { + *x = RemoveRemoteClusterResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RemoveRemoteClusterResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RemoveRemoteClusterResponse) ProtoMessage() {} + +func (x *RemoveRemoteClusterResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RemoveRemoteClusterResponse.ProtoReflect.Descriptor instead. +func (*RemoveRemoteClusterResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{11} +} + +type ListClustersRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PageSize int32 `protobuf:"varint,1,opt,name=page_size,json=pageSize,proto3" json:"page_size,omitempty"` + NextPageToken []byte `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *ListClustersRequest) Reset() { + *x = ListClustersRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListClustersRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListClustersRequest) ProtoMessage() {} + +func (x *ListClustersRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListClustersRequest.ProtoReflect.Descriptor instead. +func (*ListClustersRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{12} +} + +func (x *ListClustersRequest) GetPageSize() int32 { + if x != nil { + return x.PageSize + } + return 0 +} + +func (x *ListClustersRequest) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +type ListClustersResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // List of all cluster information + Clusters []*ClusterMetadata `protobuf:"bytes,1,rep,name=clusters,proto3" json:"clusters,omitempty"` + NextPageToken []byte `protobuf:"bytes,4,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *ListClustersResponse) Reset() { + *x = ListClustersResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListClustersResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListClustersResponse) ProtoMessage() {} + +func (x *ListClustersResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListClustersResponse.ProtoReflect.Descriptor instead. +func (*ListClustersResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{13} +} + +func (x *ListClustersResponse) GetClusters() []*ClusterMetadata { + if x != nil { + return x.Clusters + } + return nil +} + +func (x *ListClustersResponse) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +type ClusterMetadata struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Name of the cluster name. + ClusterName string `protobuf:"bytes,1,opt,name=cluster_name,json=clusterName,proto3" json:"cluster_name,omitempty"` + // Id of the cluster. + ClusterId string `protobuf:"bytes,2,opt,name=cluster_id,json=clusterId,proto3" json:"cluster_id,omitempty"` + // gRPC address. + Address string `protobuf:"bytes,3,opt,name=address,proto3" json:"address,omitempty"` + // HTTP address, if one exists. + HttpAddress string `protobuf:"bytes,7,opt,name=http_address,json=httpAddress,proto3" json:"http_address,omitempty"` + // A unique failover version across all connected clusters. + InitialFailoverVersion int64 `protobuf:"varint,4,opt,name=initial_failover_version,json=initialFailoverVersion,proto3" json:"initial_failover_version,omitempty"` + // History service shard number. + HistoryShardCount int32 `protobuf:"varint,5,opt,name=history_shard_count,json=historyShardCount,proto3" json:"history_shard_count,omitempty"` + // A flag to indicate if a connection is active. + IsConnectionEnabled bool `protobuf:"varint,6,opt,name=is_connection_enabled,json=isConnectionEnabled,proto3" json:"is_connection_enabled,omitempty"` +} + +func (x *ClusterMetadata) Reset() { + *x = ClusterMetadata{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ClusterMetadata) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ClusterMetadata) ProtoMessage() {} + +func (x *ClusterMetadata) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ClusterMetadata.ProtoReflect.Descriptor instead. +func (*ClusterMetadata) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{14} +} + +func (x *ClusterMetadata) GetClusterName() string { + if x != nil { + return x.ClusterName + } + return "" +} + +func (x *ClusterMetadata) GetClusterId() string { + if x != nil { + return x.ClusterId + } + return "" +} + +func (x *ClusterMetadata) GetAddress() string { + if x != nil { + return x.Address + } + return "" +} + +func (x *ClusterMetadata) GetHttpAddress() string { + if x != nil { + return x.HttpAddress + } + return "" +} + +func (x *ClusterMetadata) GetInitialFailoverVersion() int64 { + if x != nil { + return x.InitialFailoverVersion + } + return 0 +} + +func (x *ClusterMetadata) GetHistoryShardCount() int32 { + if x != nil { + return x.HistoryShardCount + } + return 0 +} + +func (x *ClusterMetadata) GetIsConnectionEnabled() bool { + if x != nil { + return x.IsConnectionEnabled + } + return false +} + +type GetNexusEndpointRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Server-generated unique endpoint ID. + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` +} + +func (x *GetNexusEndpointRequest) Reset() { + *x = GetNexusEndpointRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetNexusEndpointRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetNexusEndpointRequest) ProtoMessage() {} + +func (x *GetNexusEndpointRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[15] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetNexusEndpointRequest.ProtoReflect.Descriptor instead. +func (*GetNexusEndpointRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{15} +} + +func (x *GetNexusEndpointRequest) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +type GetNexusEndpointResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Endpoint *v11.Endpoint `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"` +} + +func (x *GetNexusEndpointResponse) Reset() { + *x = GetNexusEndpointResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetNexusEndpointResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetNexusEndpointResponse) ProtoMessage() {} + +func (x *GetNexusEndpointResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[16] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetNexusEndpointResponse.ProtoReflect.Descriptor instead. +func (*GetNexusEndpointResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{16} +} + +func (x *GetNexusEndpointResponse) GetEndpoint() *v11.Endpoint { + if x != nil { + return x.Endpoint + } + return nil +} + +type CreateNexusEndpointRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Endpoint definition to create. + Spec *v11.EndpointSpec `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"` +} + +func (x *CreateNexusEndpointRequest) Reset() { + *x = CreateNexusEndpointRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateNexusEndpointRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateNexusEndpointRequest) ProtoMessage() {} + +func (x *CreateNexusEndpointRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[17] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateNexusEndpointRequest.ProtoReflect.Descriptor instead. +func (*CreateNexusEndpointRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{17} +} + +func (x *CreateNexusEndpointRequest) GetSpec() *v11.EndpointSpec { + if x != nil { + return x.Spec + } + return nil +} + +type CreateNexusEndpointResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Data post acceptance. Can be used to issue additional updates to this record. + Endpoint *v11.Endpoint `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"` +} + +func (x *CreateNexusEndpointResponse) Reset() { + *x = CreateNexusEndpointResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateNexusEndpointResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateNexusEndpointResponse) ProtoMessage() {} + +func (x *CreateNexusEndpointResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[18] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateNexusEndpointResponse.ProtoReflect.Descriptor instead. +func (*CreateNexusEndpointResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{18} +} + +func (x *CreateNexusEndpointResponse) GetEndpoint() *v11.Endpoint { + if x != nil { + return x.Endpoint + } + return nil +} + +type UpdateNexusEndpointRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Server-generated unique endpoint ID. + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + // Data version for this endpoint. Must match current version. + Version int64 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"` + Spec *v11.EndpointSpec `protobuf:"bytes,3,opt,name=spec,proto3" json:"spec,omitempty"` +} + +func (x *UpdateNexusEndpointRequest) Reset() { + *x = UpdateNexusEndpointRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateNexusEndpointRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateNexusEndpointRequest) ProtoMessage() {} + +func (x *UpdateNexusEndpointRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[19] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateNexusEndpointRequest.ProtoReflect.Descriptor instead. +func (*UpdateNexusEndpointRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{19} +} + +func (x *UpdateNexusEndpointRequest) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *UpdateNexusEndpointRequest) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *UpdateNexusEndpointRequest) GetSpec() *v11.EndpointSpec { + if x != nil { + return x.Spec + } + return nil +} + +type UpdateNexusEndpointResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Data post acceptance. Can be used to issue additional updates to this record. + Endpoint *v11.Endpoint `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"` +} + +func (x *UpdateNexusEndpointResponse) Reset() { + *x = UpdateNexusEndpointResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateNexusEndpointResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateNexusEndpointResponse) ProtoMessage() {} + +func (x *UpdateNexusEndpointResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[20] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateNexusEndpointResponse.ProtoReflect.Descriptor instead. +func (*UpdateNexusEndpointResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{20} +} + +func (x *UpdateNexusEndpointResponse) GetEndpoint() *v11.Endpoint { + if x != nil { + return x.Endpoint + } + return nil +} + +type DeleteNexusEndpointRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Server-generated unique endpoint ID. + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + // Data version for this endpoint. Must match current version. + Version int64 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"` +} + +func (x *DeleteNexusEndpointRequest) Reset() { + *x = DeleteNexusEndpointRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteNexusEndpointRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteNexusEndpointRequest) ProtoMessage() {} + +func (x *DeleteNexusEndpointRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[21] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteNexusEndpointRequest.ProtoReflect.Descriptor instead. +func (*DeleteNexusEndpointRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{21} +} + +func (x *DeleteNexusEndpointRequest) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *DeleteNexusEndpointRequest) GetVersion() int64 { + if x != nil { + return x.Version + } + return 0 +} + +type DeleteNexusEndpointResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *DeleteNexusEndpointResponse) Reset() { + *x = DeleteNexusEndpointResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteNexusEndpointResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteNexusEndpointResponse) ProtoMessage() {} + +func (x *DeleteNexusEndpointResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[22] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteNexusEndpointResponse.ProtoReflect.Descriptor instead. +func (*DeleteNexusEndpointResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{22} +} + +type ListNexusEndpointsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PageSize int32 `protobuf:"varint,1,opt,name=page_size,json=pageSize,proto3" json:"page_size,omitempty"` + // To get the next page, pass in `ListNexusEndpointsResponse.next_page_token` from the previous page's + // response, the token will be empty if there's no other page. + // Note: the last page may be empty if the total number of endpoints registered is a multiple of the page size. + NextPageToken []byte `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` + // Name of the incoming endpoint to filter on - optional. Specifying this will result in zero or one results. + // (-- api-linter: core::203::field-behavior-required=disabled + // + // aip.dev/not-precedent: Not following linter rules. --) + Name string `protobuf:"bytes,3,opt,name=name,proto3" json:"name,omitempty"` +} + +func (x *ListNexusEndpointsRequest) Reset() { + *x = ListNexusEndpointsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListNexusEndpointsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListNexusEndpointsRequest) ProtoMessage() {} + +func (x *ListNexusEndpointsRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[23] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListNexusEndpointsRequest.ProtoReflect.Descriptor instead. +func (*ListNexusEndpointsRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{23} +} + +func (x *ListNexusEndpointsRequest) GetPageSize() int32 { + if x != nil { + return x.PageSize + } + return 0 +} + +func (x *ListNexusEndpointsRequest) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +func (x *ListNexusEndpointsRequest) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type ListNexusEndpointsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Token for getting the next page. + NextPageToken []byte `protobuf:"bytes,1,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` + Endpoints []*v11.Endpoint `protobuf:"bytes,2,rep,name=endpoints,proto3" json:"endpoints,omitempty"` +} + +func (x *ListNexusEndpointsResponse) Reset() { + *x = ListNexusEndpointsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListNexusEndpointsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListNexusEndpointsResponse) ProtoMessage() {} + +func (x *ListNexusEndpointsResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[24] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListNexusEndpointsResponse.ProtoReflect.Descriptor instead. +func (*ListNexusEndpointsResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP(), []int{24} +} + +func (x *ListNexusEndpointsResponse) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +func (x *ListNexusEndpointsResponse) GetEndpoints() []*v11.Endpoint { + if x != nil { + return x.Endpoints + } + return nil +} + +var File_temporal_api_operatorservice_v1_request_response_proto protoreflect.FileDescriptor + +var file_temporal_api_operatorservice_v1_request_response_proto_rawDesc = []byte{ + 0x0a, 0x36, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x6f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, 0x76, 0x31, 0x2f, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x1f, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x1a, 0x22, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x23, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xb9, 0x02, 0x0a, 0x1a, 0x41, 0x64, 0x64, 0x53, + 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x82, 0x01, 0x0a, 0x11, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, + 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x51, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x41, 0x64, 0x64, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, + 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x1a, 0x74, 0x0a, 0x15, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x41, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, + 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x1d, 0x0a, 0x1b, 0x41, 0x64, 0x64, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x72, 0x0a, 0x1d, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x2f, 0x0a, 0x11, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x20, + 0x0a, 0x1e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3f, + 0x0a, 0x1b, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xdf, 0x05, 0x0a, 0x1c, + 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x84, 0x01, 0x0a, 0x11, + 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x53, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x2e, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x84, 0x01, 0x0a, 0x11, 0x73, + 0x79, 0x73, 0x74, 0x65, 0x6d, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x53, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x2e, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x7b, 0x0a, 0x0e, 0x73, 0x74, + 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x50, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x74, + 0x6f, 0x72, 0x61, 0x67, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x02, 0x68, + 0x00, 0x1a, 0x74, 0x0a, 0x15, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, + 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x54, + 0x79, 0x70, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, + 0x01, 0x1a, 0x74, 0x0a, 0x15, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x54, 0x79, + 0x70, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, + 0x1a, 0x48, 0x0a, 0x12, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xb6, 0x01, 0x0a, 0x16, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x25, 0x0a, 0x0c, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x53, 0x0a, 0x16, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x14, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x44, 0x65, 0x6c, + 0x61, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x4a, 0x0a, 0x17, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x2f, 0x0a, 0x11, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x64, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd5, + 0x01, 0x0a, 0x1f, 0x41, 0x64, 0x64, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x6d, + 0x6f, 0x74, 0x65, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x2d, 0x0a, 0x10, 0x66, 0x72, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x64, 0x5f, 0x61, 0x64, 0x64, 0x72, + 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x66, 0x72, 0x6f, 0x6e, 0x74, 0x65, + 0x6e, 0x64, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x20, + 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x5f, 0x63, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x1d, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74, + 0x65, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x36, 0x0a, 0x15, 0x66, 0x72, 0x6f, 0x6e, 0x74, 0x65, 0x6e, + 0x64, 0x5f, 0x68, 0x74, 0x74, 0x70, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x13, 0x66, 0x72, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x64, 0x48, 0x74, 0x74, 0x70, + 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x22, 0x0a, 0x20, 0x41, 0x64, + 0x64, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x43, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x43, 0x0a, 0x1a, + 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x43, 0x6c, 0x75, 0x73, 0x74, + 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, 0x0c, 0x63, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x1d, + 0x0a, 0x1b, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x43, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x62, 0x0a, 0x13, 0x4c, + 0x69, 0x73, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x1f, 0x0a, 0x09, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, + 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, + 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x94, 0x01, 0x0a, 0x14, 0x4c, + 0x69, 0x73, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x50, 0x0a, 0x08, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x52, 0x08, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, + 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, + 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xca, 0x02, 0x0a, 0x0f, 0x43, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x25, 0x0a, 0x0c, + 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, + 0x0a, 0x0c, 0x68, 0x74, 0x74, 0x70, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0b, 0x68, 0x74, 0x74, 0x70, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x3c, 0x0a, 0x18, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x66, + 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x16, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x46, 0x61, 0x69, 0x6c, + 0x6f, 0x76, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x32, + 0x0a, 0x13, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x63, + 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x11, 0x68, 0x69, 0x73, 0x74, 0x6f, + 0x72, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x36, 0x0a, 0x15, 0x69, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x13, 0x69, 0x73, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x61, 0x62, 0x6c, 0x65, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x2d, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x4e, 0x65, 0x78, 0x75, 0x73, + 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, + 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x5b, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3f, 0x0a, 0x08, 0x65, + 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, + 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x08, 0x65, 0x6e, 0x64, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0x59, 0x0a, 0x1a, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3b, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x42, 0x02, 0x68, 0x00, 0x22, + 0x5e, 0x0a, 0x1b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3f, 0x0a, 0x08, + 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, + 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x08, 0x65, + 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8b, 0x01, 0x0a, 0x1a, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3b, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x64, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x5e, 0x0a, 0x1b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, + 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3f, + 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, + 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x08, + 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0x4e, 0x0a, 0x1a, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x1d, 0x0a, 0x1b, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x80, 0x01, 0x0a, 0x19, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x65, 0x78, + 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x1f, 0x0a, 0x09, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, + 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, + 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x16, 0x0a, 0x04, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x8b, 0x01, 0x0a, 0x1a, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, + 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, + 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, + 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x09, 0x65, 0x6e, 0x64, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, + 0x2e, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x09, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x73, 0x42, 0x02, 0x68, 0x00, 0x42, 0xbe, 0x01, 0x0a, 0x22, 0x69, 0x6f, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x14, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x74, + 0x6f, 0x50, 0x01, 0x5a, 0x35, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, 0x76, 0x31, 0x3b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0xaa, 0x02, 0x21, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x53, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x24, 0x54, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x6f, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_operatorservice_v1_request_response_proto_rawDescOnce sync.Once + file_temporal_api_operatorservice_v1_request_response_proto_rawDescData = file_temporal_api_operatorservice_v1_request_response_proto_rawDesc +) + +func file_temporal_api_operatorservice_v1_request_response_proto_rawDescGZIP() []byte { + file_temporal_api_operatorservice_v1_request_response_proto_rawDescOnce.Do(func() { + file_temporal_api_operatorservice_v1_request_response_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_operatorservice_v1_request_response_proto_rawDescData) + }) + return file_temporal_api_operatorservice_v1_request_response_proto_rawDescData +} + +var file_temporal_api_operatorservice_v1_request_response_proto_msgTypes = make([]protoimpl.MessageInfo, 29) +var file_temporal_api_operatorservice_v1_request_response_proto_goTypes = []any{ + (*AddSearchAttributesRequest)(nil), // 0: temporal.api.operatorservice.v1.AddSearchAttributesRequest + (*AddSearchAttributesResponse)(nil), // 1: temporal.api.operatorservice.v1.AddSearchAttributesResponse + (*RemoveSearchAttributesRequest)(nil), // 2: temporal.api.operatorservice.v1.RemoveSearchAttributesRequest + (*RemoveSearchAttributesResponse)(nil), // 3: temporal.api.operatorservice.v1.RemoveSearchAttributesResponse + (*ListSearchAttributesRequest)(nil), // 4: temporal.api.operatorservice.v1.ListSearchAttributesRequest + (*ListSearchAttributesResponse)(nil), // 5: temporal.api.operatorservice.v1.ListSearchAttributesResponse + (*DeleteNamespaceRequest)(nil), // 6: temporal.api.operatorservice.v1.DeleteNamespaceRequest + (*DeleteNamespaceResponse)(nil), // 7: temporal.api.operatorservice.v1.DeleteNamespaceResponse + (*AddOrUpdateRemoteClusterRequest)(nil), // 8: temporal.api.operatorservice.v1.AddOrUpdateRemoteClusterRequest + (*AddOrUpdateRemoteClusterResponse)(nil), // 9: temporal.api.operatorservice.v1.AddOrUpdateRemoteClusterResponse + (*RemoveRemoteClusterRequest)(nil), // 10: temporal.api.operatorservice.v1.RemoveRemoteClusterRequest + (*RemoveRemoteClusterResponse)(nil), // 11: temporal.api.operatorservice.v1.RemoveRemoteClusterResponse + (*ListClustersRequest)(nil), // 12: temporal.api.operatorservice.v1.ListClustersRequest + (*ListClustersResponse)(nil), // 13: temporal.api.operatorservice.v1.ListClustersResponse + (*ClusterMetadata)(nil), // 14: temporal.api.operatorservice.v1.ClusterMetadata + (*GetNexusEndpointRequest)(nil), // 15: temporal.api.operatorservice.v1.GetNexusEndpointRequest + (*GetNexusEndpointResponse)(nil), // 16: temporal.api.operatorservice.v1.GetNexusEndpointResponse + (*CreateNexusEndpointRequest)(nil), // 17: temporal.api.operatorservice.v1.CreateNexusEndpointRequest + (*CreateNexusEndpointResponse)(nil), // 18: temporal.api.operatorservice.v1.CreateNexusEndpointResponse + (*UpdateNexusEndpointRequest)(nil), // 19: temporal.api.operatorservice.v1.UpdateNexusEndpointRequest + (*UpdateNexusEndpointResponse)(nil), // 20: temporal.api.operatorservice.v1.UpdateNexusEndpointResponse + (*DeleteNexusEndpointRequest)(nil), // 21: temporal.api.operatorservice.v1.DeleteNexusEndpointRequest + (*DeleteNexusEndpointResponse)(nil), // 22: temporal.api.operatorservice.v1.DeleteNexusEndpointResponse + (*ListNexusEndpointsRequest)(nil), // 23: temporal.api.operatorservice.v1.ListNexusEndpointsRequest + (*ListNexusEndpointsResponse)(nil), // 24: temporal.api.operatorservice.v1.ListNexusEndpointsResponse + nil, // 25: temporal.api.operatorservice.v1.AddSearchAttributesRequest.SearchAttributesEntry + nil, // 26: temporal.api.operatorservice.v1.ListSearchAttributesResponse.CustomAttributesEntry + nil, // 27: temporal.api.operatorservice.v1.ListSearchAttributesResponse.SystemAttributesEntry + nil, // 28: temporal.api.operatorservice.v1.ListSearchAttributesResponse.StorageSchemaEntry + (*durationpb.Duration)(nil), // 29: google.protobuf.Duration + (*v11.Endpoint)(nil), // 30: temporal.api.nexus.v1.Endpoint + (*v11.EndpointSpec)(nil), // 31: temporal.api.nexus.v1.EndpointSpec + (v1.IndexedValueType)(0), // 32: temporal.api.enums.v1.IndexedValueType +} +var file_temporal_api_operatorservice_v1_request_response_proto_depIdxs = []int32{ + 25, // 0: temporal.api.operatorservice.v1.AddSearchAttributesRequest.search_attributes:type_name -> temporal.api.operatorservice.v1.AddSearchAttributesRequest.SearchAttributesEntry + 26, // 1: temporal.api.operatorservice.v1.ListSearchAttributesResponse.custom_attributes:type_name -> temporal.api.operatorservice.v1.ListSearchAttributesResponse.CustomAttributesEntry + 27, // 2: temporal.api.operatorservice.v1.ListSearchAttributesResponse.system_attributes:type_name -> temporal.api.operatorservice.v1.ListSearchAttributesResponse.SystemAttributesEntry + 28, // 3: temporal.api.operatorservice.v1.ListSearchAttributesResponse.storage_schema:type_name -> temporal.api.operatorservice.v1.ListSearchAttributesResponse.StorageSchemaEntry + 29, // 4: temporal.api.operatorservice.v1.DeleteNamespaceRequest.namespace_delete_delay:type_name -> google.protobuf.Duration + 14, // 5: temporal.api.operatorservice.v1.ListClustersResponse.clusters:type_name -> temporal.api.operatorservice.v1.ClusterMetadata + 30, // 6: temporal.api.operatorservice.v1.GetNexusEndpointResponse.endpoint:type_name -> temporal.api.nexus.v1.Endpoint + 31, // 7: temporal.api.operatorservice.v1.CreateNexusEndpointRequest.spec:type_name -> temporal.api.nexus.v1.EndpointSpec + 30, // 8: temporal.api.operatorservice.v1.CreateNexusEndpointResponse.endpoint:type_name -> temporal.api.nexus.v1.Endpoint + 31, // 9: temporal.api.operatorservice.v1.UpdateNexusEndpointRequest.spec:type_name -> temporal.api.nexus.v1.EndpointSpec + 30, // 10: temporal.api.operatorservice.v1.UpdateNexusEndpointResponse.endpoint:type_name -> temporal.api.nexus.v1.Endpoint + 30, // 11: temporal.api.operatorservice.v1.ListNexusEndpointsResponse.endpoints:type_name -> temporal.api.nexus.v1.Endpoint + 32, // 12: temporal.api.operatorservice.v1.AddSearchAttributesRequest.SearchAttributesEntry.value:type_name -> temporal.api.enums.v1.IndexedValueType + 32, // 13: temporal.api.operatorservice.v1.ListSearchAttributesResponse.CustomAttributesEntry.value:type_name -> temporal.api.enums.v1.IndexedValueType + 32, // 14: temporal.api.operatorservice.v1.ListSearchAttributesResponse.SystemAttributesEntry.value:type_name -> temporal.api.enums.v1.IndexedValueType + 15, // [15:15] is the sub-list for method output_type + 15, // [15:15] is the sub-list for method input_type + 15, // [15:15] is the sub-list for extension type_name + 15, // [15:15] is the sub-list for extension extendee + 0, // [0:15] is the sub-list for field type_name +} + +func init() { file_temporal_api_operatorservice_v1_request_response_proto_init() } +func file_temporal_api_operatorservice_v1_request_response_proto_init() { + if File_temporal_api_operatorservice_v1_request_response_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*AddSearchAttributesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*AddSearchAttributesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*RemoveSearchAttributesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*RemoveSearchAttributesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*ListSearchAttributesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*ListSearchAttributesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*DeleteNamespaceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*DeleteNamespaceResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*AddOrUpdateRemoteClusterRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*AddOrUpdateRemoteClusterResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*RemoveRemoteClusterRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[11].Exporter = func(v any, i int) any { + switch v := v.(*RemoveRemoteClusterResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[12].Exporter = func(v any, i int) any { + switch v := v.(*ListClustersRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[13].Exporter = func(v any, i int) any { + switch v := v.(*ListClustersResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[14].Exporter = func(v any, i int) any { + switch v := v.(*ClusterMetadata); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[15].Exporter = func(v any, i int) any { + switch v := v.(*GetNexusEndpointRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[16].Exporter = func(v any, i int) any { + switch v := v.(*GetNexusEndpointResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[17].Exporter = func(v any, i int) any { + switch v := v.(*CreateNexusEndpointRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[18].Exporter = func(v any, i int) any { + switch v := v.(*CreateNexusEndpointResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[19].Exporter = func(v any, i int) any { + switch v := v.(*UpdateNexusEndpointRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[20].Exporter = func(v any, i int) any { + switch v := v.(*UpdateNexusEndpointResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[21].Exporter = func(v any, i int) any { + switch v := v.(*DeleteNexusEndpointRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[22].Exporter = func(v any, i int) any { + switch v := v.(*DeleteNexusEndpointResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[23].Exporter = func(v any, i int) any { + switch v := v.(*ListNexusEndpointsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_operatorservice_v1_request_response_proto_msgTypes[24].Exporter = func(v any, i int) any { + switch v := v.(*ListNexusEndpointsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_operatorservice_v1_request_response_proto_rawDesc, + NumEnums: 0, + NumMessages: 29, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_operatorservice_v1_request_response_proto_goTypes, + DependencyIndexes: file_temporal_api_operatorservice_v1_request_response_proto_depIdxs, + MessageInfos: file_temporal_api_operatorservice_v1_request_response_proto_msgTypes, + }.Build() + File_temporal_api_operatorservice_v1_request_response_proto = out.File + file_temporal_api_operatorservice_v1_request_response_proto_rawDesc = nil + file_temporal_api_operatorservice_v1_request_response_proto_goTypes = nil + file_temporal_api_operatorservice_v1_request_response_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/operatorservice/v1/service.pb.go b/vendor/go.temporal.io/api/operatorservice/v1/service.pb.go new file mode 100644 index 00000000000..d8c3a2877c1 --- /dev/null +++ b/vendor/go.temporal.io/api/operatorservice/v1/service.pb.go @@ -0,0 +1,295 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/operatorservice/v1/service.proto + +package operatorservice + +import ( + reflect "reflect" + + _ "google.golang.org/genproto/googleapis/api/annotations" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +var File_temporal_api_operatorservice_v1_service_proto protoreflect.FileDescriptor + +var file_temporal_api_operatorservice_v1_service_proto_rawDesc = []byte{ + 0x0a, 0x2d, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, 0x76, + 0x31, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, + 0x1f, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, + 0x1a, 0x36, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, 0x76, + 0x31, 0x2f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x32, 0xc6, 0x11, 0x0a, 0x0f, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x6f, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x92, 0x01, 0x0a, 0x13, 0x41, + 0x64, 0x64, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x12, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x64, 0x64, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x3c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x41, 0x64, 0x64, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x9b, 0x01, 0x0a, 0x16, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x3e, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6d, + 0x6f, 0x76, 0x65, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3f, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6d, + 0x6f, 0x76, 0x65, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x82, 0x02, + 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x3c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x61, + 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x61, 0x72, 0x63, + 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x6d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x67, 0x5a, 0x32, 0x12, 0x30, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x73, 0x65, + 0x61, 0x72, 0x63, 0x68, 0x2d, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, + 0x31, 0x2f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, + 0x2f, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x2d, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x12, 0x86, 0x01, 0x0a, 0x0f, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x37, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x38, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0xa1, 0x01, 0x0a, 0x18, + 0x41, 0x64, 0x64, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74, + 0x65, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x40, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x64, 0x64, 0x4f, 0x72, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x43, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x41, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x64, 0x64, + 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x43, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x92, 0x01, 0x0a, 0x13, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, + 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, + 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x6d, + 0x6f, 0x74, 0x65, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x7d, 0x0a, 0x0c, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x73, 0x12, 0x34, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, + 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x35, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, + 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0xce, 0x01, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x4e, 0x65, 0x78, 0x75, 0x73, + 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x38, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x65, + 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x39, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x45, 0x82, + 0xd3, 0xe4, 0x93, 0x02, 0x3f, 0x5a, 0x1e, 0x12, 0x1c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, + 0x2f, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, + 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0x1d, 0x2f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2f, + 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2f, + 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xd3, 0x01, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4e, + 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x3b, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3c, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, + 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x41, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x3b, 0x3a, + 0x01, 0x2a, 0x5a, 0x1c, 0x3a, 0x01, 0x2a, 0x22, 0x17, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, + 0x2f, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, + 0x22, 0x18, 0x2f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2f, 0x6e, 0x65, 0x78, 0x75, 0x73, + 0x2f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x12, 0xeb, 0x01, 0x0a, 0x13, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x12, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, + 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x3c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x59, 0x82, + 0xd3, 0xe4, 0x93, 0x02, 0x53, 0x3a, 0x01, 0x2a, 0x5a, 0x28, 0x3a, 0x01, 0x2a, 0x22, 0x23, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2f, 0x65, 0x6e, 0x64, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x75, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x22, 0x24, 0x2f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2f, 0x6e, 0x65, 0x78, + 0x75, 0x73, 0x2f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2f, 0x7b, 0x69, 0x64, + 0x7d, 0x2f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0xd7, 0x01, 0x0a, 0x13, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, + 0x12, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, + 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3c, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x45, 0x82, 0xd3, 0xe4, + 0x93, 0x02, 0x3f, 0x5a, 0x1e, 0x2a, 0x1c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, + 0x65, 0x78, 0x75, 0x73, 0x2f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2f, 0x7b, + 0x69, 0x64, 0x7d, 0x2a, 0x1d, 0x2f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2f, 0x6e, 0x65, + 0x78, 0x75, 0x73, 0x2f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2f, 0x7b, 0x69, + 0x64, 0x7d, 0x12, 0xca, 0x01, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x65, 0x78, 0x75, 0x73, + 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x12, 0x3a, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, + 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, + 0x4e, 0x65, 0x78, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x65, 0x78, 0x75, + 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x3b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x35, 0x5a, 0x19, 0x12, 0x17, 0x2f, 0x61, + 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2f, 0x65, 0x6e, 0x64, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x12, 0x18, 0x2f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2f, + 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x42, + 0xb6, 0x01, 0x0a, 0x22, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x50, + 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x35, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, 0x76, 0x31, 0x3b, 0x6f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0xaa, 0x02, 0x21, + 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x4f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x56, + 0x31, 0xea, 0x02, 0x24, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, + 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x53, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var file_temporal_api_operatorservice_v1_service_proto_goTypes = []any{ + (*AddSearchAttributesRequest)(nil), // 0: temporal.api.operatorservice.v1.AddSearchAttributesRequest + (*RemoveSearchAttributesRequest)(nil), // 1: temporal.api.operatorservice.v1.RemoveSearchAttributesRequest + (*ListSearchAttributesRequest)(nil), // 2: temporal.api.operatorservice.v1.ListSearchAttributesRequest + (*DeleteNamespaceRequest)(nil), // 3: temporal.api.operatorservice.v1.DeleteNamespaceRequest + (*AddOrUpdateRemoteClusterRequest)(nil), // 4: temporal.api.operatorservice.v1.AddOrUpdateRemoteClusterRequest + (*RemoveRemoteClusterRequest)(nil), // 5: temporal.api.operatorservice.v1.RemoveRemoteClusterRequest + (*ListClustersRequest)(nil), // 6: temporal.api.operatorservice.v1.ListClustersRequest + (*GetNexusEndpointRequest)(nil), // 7: temporal.api.operatorservice.v1.GetNexusEndpointRequest + (*CreateNexusEndpointRequest)(nil), // 8: temporal.api.operatorservice.v1.CreateNexusEndpointRequest + (*UpdateNexusEndpointRequest)(nil), // 9: temporal.api.operatorservice.v1.UpdateNexusEndpointRequest + (*DeleteNexusEndpointRequest)(nil), // 10: temporal.api.operatorservice.v1.DeleteNexusEndpointRequest + (*ListNexusEndpointsRequest)(nil), // 11: temporal.api.operatorservice.v1.ListNexusEndpointsRequest + (*AddSearchAttributesResponse)(nil), // 12: temporal.api.operatorservice.v1.AddSearchAttributesResponse + (*RemoveSearchAttributesResponse)(nil), // 13: temporal.api.operatorservice.v1.RemoveSearchAttributesResponse + (*ListSearchAttributesResponse)(nil), // 14: temporal.api.operatorservice.v1.ListSearchAttributesResponse + (*DeleteNamespaceResponse)(nil), // 15: temporal.api.operatorservice.v1.DeleteNamespaceResponse + (*AddOrUpdateRemoteClusterResponse)(nil), // 16: temporal.api.operatorservice.v1.AddOrUpdateRemoteClusterResponse + (*RemoveRemoteClusterResponse)(nil), // 17: temporal.api.operatorservice.v1.RemoveRemoteClusterResponse + (*ListClustersResponse)(nil), // 18: temporal.api.operatorservice.v1.ListClustersResponse + (*GetNexusEndpointResponse)(nil), // 19: temporal.api.operatorservice.v1.GetNexusEndpointResponse + (*CreateNexusEndpointResponse)(nil), // 20: temporal.api.operatorservice.v1.CreateNexusEndpointResponse + (*UpdateNexusEndpointResponse)(nil), // 21: temporal.api.operatorservice.v1.UpdateNexusEndpointResponse + (*DeleteNexusEndpointResponse)(nil), // 22: temporal.api.operatorservice.v1.DeleteNexusEndpointResponse + (*ListNexusEndpointsResponse)(nil), // 23: temporal.api.operatorservice.v1.ListNexusEndpointsResponse +} +var file_temporal_api_operatorservice_v1_service_proto_depIdxs = []int32{ + 0, // 0: temporal.api.operatorservice.v1.OperatorService.AddSearchAttributes:input_type -> temporal.api.operatorservice.v1.AddSearchAttributesRequest + 1, // 1: temporal.api.operatorservice.v1.OperatorService.RemoveSearchAttributes:input_type -> temporal.api.operatorservice.v1.RemoveSearchAttributesRequest + 2, // 2: temporal.api.operatorservice.v1.OperatorService.ListSearchAttributes:input_type -> temporal.api.operatorservice.v1.ListSearchAttributesRequest + 3, // 3: temporal.api.operatorservice.v1.OperatorService.DeleteNamespace:input_type -> temporal.api.operatorservice.v1.DeleteNamespaceRequest + 4, // 4: temporal.api.operatorservice.v1.OperatorService.AddOrUpdateRemoteCluster:input_type -> temporal.api.operatorservice.v1.AddOrUpdateRemoteClusterRequest + 5, // 5: temporal.api.operatorservice.v1.OperatorService.RemoveRemoteCluster:input_type -> temporal.api.operatorservice.v1.RemoveRemoteClusterRequest + 6, // 6: temporal.api.operatorservice.v1.OperatorService.ListClusters:input_type -> temporal.api.operatorservice.v1.ListClustersRequest + 7, // 7: temporal.api.operatorservice.v1.OperatorService.GetNexusEndpoint:input_type -> temporal.api.operatorservice.v1.GetNexusEndpointRequest + 8, // 8: temporal.api.operatorservice.v1.OperatorService.CreateNexusEndpoint:input_type -> temporal.api.operatorservice.v1.CreateNexusEndpointRequest + 9, // 9: temporal.api.operatorservice.v1.OperatorService.UpdateNexusEndpoint:input_type -> temporal.api.operatorservice.v1.UpdateNexusEndpointRequest + 10, // 10: temporal.api.operatorservice.v1.OperatorService.DeleteNexusEndpoint:input_type -> temporal.api.operatorservice.v1.DeleteNexusEndpointRequest + 11, // 11: temporal.api.operatorservice.v1.OperatorService.ListNexusEndpoints:input_type -> temporal.api.operatorservice.v1.ListNexusEndpointsRequest + 12, // 12: temporal.api.operatorservice.v1.OperatorService.AddSearchAttributes:output_type -> temporal.api.operatorservice.v1.AddSearchAttributesResponse + 13, // 13: temporal.api.operatorservice.v1.OperatorService.RemoveSearchAttributes:output_type -> temporal.api.operatorservice.v1.RemoveSearchAttributesResponse + 14, // 14: temporal.api.operatorservice.v1.OperatorService.ListSearchAttributes:output_type -> temporal.api.operatorservice.v1.ListSearchAttributesResponse + 15, // 15: temporal.api.operatorservice.v1.OperatorService.DeleteNamespace:output_type -> temporal.api.operatorservice.v1.DeleteNamespaceResponse + 16, // 16: temporal.api.operatorservice.v1.OperatorService.AddOrUpdateRemoteCluster:output_type -> temporal.api.operatorservice.v1.AddOrUpdateRemoteClusterResponse + 17, // 17: temporal.api.operatorservice.v1.OperatorService.RemoveRemoteCluster:output_type -> temporal.api.operatorservice.v1.RemoveRemoteClusterResponse + 18, // 18: temporal.api.operatorservice.v1.OperatorService.ListClusters:output_type -> temporal.api.operatorservice.v1.ListClustersResponse + 19, // 19: temporal.api.operatorservice.v1.OperatorService.GetNexusEndpoint:output_type -> temporal.api.operatorservice.v1.GetNexusEndpointResponse + 20, // 20: temporal.api.operatorservice.v1.OperatorService.CreateNexusEndpoint:output_type -> temporal.api.operatorservice.v1.CreateNexusEndpointResponse + 21, // 21: temporal.api.operatorservice.v1.OperatorService.UpdateNexusEndpoint:output_type -> temporal.api.operatorservice.v1.UpdateNexusEndpointResponse + 22, // 22: temporal.api.operatorservice.v1.OperatorService.DeleteNexusEndpoint:output_type -> temporal.api.operatorservice.v1.DeleteNexusEndpointResponse + 23, // 23: temporal.api.operatorservice.v1.OperatorService.ListNexusEndpoints:output_type -> temporal.api.operatorservice.v1.ListNexusEndpointsResponse + 12, // [12:24] is the sub-list for method output_type + 0, // [0:12] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_operatorservice_v1_service_proto_init() } +func file_temporal_api_operatorservice_v1_service_proto_init() { + if File_temporal_api_operatorservice_v1_service_proto != nil { + return + } + file_temporal_api_operatorservice_v1_request_response_proto_init() + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_operatorservice_v1_service_proto_rawDesc, + NumEnums: 0, + NumMessages: 0, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_temporal_api_operatorservice_v1_service_proto_goTypes, + DependencyIndexes: file_temporal_api_operatorservice_v1_service_proto_depIdxs, + }.Build() + File_temporal_api_operatorservice_v1_service_proto = out.File + file_temporal_api_operatorservice_v1_service_proto_rawDesc = nil + file_temporal_api_operatorservice_v1_service_proto_goTypes = nil + file_temporal_api_operatorservice_v1_service_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/operatorservice/v1/service.pb.gw.go b/vendor/go.temporal.io/api/operatorservice/v1/service.pb.gw.go new file mode 100644 index 00000000000..6ff8c2b1609 --- /dev/null +++ b/vendor/go.temporal.io/api/operatorservice/v1/service.pb.gw.go @@ -0,0 +1,1313 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-grpc-gateway. DO NOT EDIT. +// source: temporal/api/operatorservice/v1/service.proto + +/* +Package operatorservice is a reverse proxy. + +It translates gRPC into RESTful JSON APIs. +*/ +package operatorservice + +import ( + "context" + "io" + "net/http" + + "github.com/grpc-ecosystem/grpc-gateway/v2/runtime" + "github.com/grpc-ecosystem/grpc-gateway/v2/utilities" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/grpclog" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/proto" +) + +// Suppress "imported and not used" errors +var _ codes.Code +var _ io.Reader +var _ status.Status +var _ = runtime.String +var _ = utilities.NewDoubleArray +var _ = metadata.Join + +func request_OperatorService_ListSearchAttributes_0(ctx context.Context, marshaler runtime.Marshaler, client OperatorServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListSearchAttributesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.ListSearchAttributes(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_OperatorService_ListSearchAttributes_0(ctx context.Context, marshaler runtime.Marshaler, server OperatorServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListSearchAttributesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.ListSearchAttributes(ctx, &protoReq) + return msg, metadata, err + +} + +func request_OperatorService_ListSearchAttributes_1(ctx context.Context, marshaler runtime.Marshaler, client OperatorServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListSearchAttributesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.ListSearchAttributes(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_OperatorService_ListSearchAttributes_1(ctx context.Context, marshaler runtime.Marshaler, server OperatorServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListSearchAttributesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.ListSearchAttributes(ctx, &protoReq) + return msg, metadata, err + +} + +func request_OperatorService_GetNexusEndpoint_0(ctx context.Context, marshaler runtime.Marshaler, client OperatorServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetNexusEndpointRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + } + + protoReq.Id, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + } + + msg, err := client.GetNexusEndpoint(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_OperatorService_GetNexusEndpoint_0(ctx context.Context, marshaler runtime.Marshaler, server OperatorServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetNexusEndpointRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + } + + protoReq.Id, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + } + + msg, err := server.GetNexusEndpoint(ctx, &protoReq) + return msg, metadata, err + +} + +func request_OperatorService_GetNexusEndpoint_1(ctx context.Context, marshaler runtime.Marshaler, client OperatorServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetNexusEndpointRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + } + + protoReq.Id, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + } + + msg, err := client.GetNexusEndpoint(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_OperatorService_GetNexusEndpoint_1(ctx context.Context, marshaler runtime.Marshaler, server OperatorServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetNexusEndpointRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + } + + protoReq.Id, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + } + + msg, err := server.GetNexusEndpoint(ctx, &protoReq) + return msg, metadata, err + +} + +func request_OperatorService_CreateNexusEndpoint_0(ctx context.Context, marshaler runtime.Marshaler, client OperatorServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateNexusEndpointRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.CreateNexusEndpoint(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_OperatorService_CreateNexusEndpoint_0(ctx context.Context, marshaler runtime.Marshaler, server OperatorServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateNexusEndpointRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.CreateNexusEndpoint(ctx, &protoReq) + return msg, metadata, err + +} + +func request_OperatorService_CreateNexusEndpoint_1(ctx context.Context, marshaler runtime.Marshaler, client OperatorServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateNexusEndpointRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.CreateNexusEndpoint(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_OperatorService_CreateNexusEndpoint_1(ctx context.Context, marshaler runtime.Marshaler, server OperatorServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateNexusEndpointRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.CreateNexusEndpoint(ctx, &protoReq) + return msg, metadata, err + +} + +func request_OperatorService_UpdateNexusEndpoint_0(ctx context.Context, marshaler runtime.Marshaler, client OperatorServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateNexusEndpointRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + } + + protoReq.Id, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + } + + msg, err := client.UpdateNexusEndpoint(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_OperatorService_UpdateNexusEndpoint_0(ctx context.Context, marshaler runtime.Marshaler, server OperatorServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateNexusEndpointRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + } + + protoReq.Id, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + } + + msg, err := server.UpdateNexusEndpoint(ctx, &protoReq) + return msg, metadata, err + +} + +func request_OperatorService_UpdateNexusEndpoint_1(ctx context.Context, marshaler runtime.Marshaler, client OperatorServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateNexusEndpointRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + } + + protoReq.Id, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + } + + msg, err := client.UpdateNexusEndpoint(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_OperatorService_UpdateNexusEndpoint_1(ctx context.Context, marshaler runtime.Marshaler, server OperatorServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateNexusEndpointRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + } + + protoReq.Id, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + } + + msg, err := server.UpdateNexusEndpoint(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_OperatorService_DeleteNexusEndpoint_0 = &utilities.DoubleArray{Encoding: map[string]int{"id": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_OperatorService_DeleteNexusEndpoint_0(ctx context.Context, marshaler runtime.Marshaler, client OperatorServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteNexusEndpointRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + } + + protoReq.Id, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_OperatorService_DeleteNexusEndpoint_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DeleteNexusEndpoint(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_OperatorService_DeleteNexusEndpoint_0(ctx context.Context, marshaler runtime.Marshaler, server OperatorServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteNexusEndpointRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + } + + protoReq.Id, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_OperatorService_DeleteNexusEndpoint_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DeleteNexusEndpoint(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_OperatorService_DeleteNexusEndpoint_1 = &utilities.DoubleArray{Encoding: map[string]int{"id": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_OperatorService_DeleteNexusEndpoint_1(ctx context.Context, marshaler runtime.Marshaler, client OperatorServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteNexusEndpointRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + } + + protoReq.Id, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_OperatorService_DeleteNexusEndpoint_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DeleteNexusEndpoint(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_OperatorService_DeleteNexusEndpoint_1(ctx context.Context, marshaler runtime.Marshaler, server OperatorServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteNexusEndpointRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + } + + protoReq.Id, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_OperatorService_DeleteNexusEndpoint_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DeleteNexusEndpoint(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_OperatorService_ListNexusEndpoints_0 = &utilities.DoubleArray{Encoding: map[string]int{}, Base: []int(nil), Check: []int(nil)} +) + +func request_OperatorService_ListNexusEndpoints_0(ctx context.Context, marshaler runtime.Marshaler, client OperatorServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListNexusEndpointsRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_OperatorService_ListNexusEndpoints_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListNexusEndpoints(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_OperatorService_ListNexusEndpoints_0(ctx context.Context, marshaler runtime.Marshaler, server OperatorServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListNexusEndpointsRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_OperatorService_ListNexusEndpoints_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListNexusEndpoints(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_OperatorService_ListNexusEndpoints_1 = &utilities.DoubleArray{Encoding: map[string]int{}, Base: []int(nil), Check: []int(nil)} +) + +func request_OperatorService_ListNexusEndpoints_1(ctx context.Context, marshaler runtime.Marshaler, client OperatorServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListNexusEndpointsRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_OperatorService_ListNexusEndpoints_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListNexusEndpoints(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_OperatorService_ListNexusEndpoints_1(ctx context.Context, marshaler runtime.Marshaler, server OperatorServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListNexusEndpointsRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_OperatorService_ListNexusEndpoints_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListNexusEndpoints(ctx, &protoReq) + return msg, metadata, err + +} + +// RegisterOperatorServiceHandlerServer registers the http handlers for service OperatorService to "mux". +// UnaryRPC :call OperatorServiceServer directly. +// StreamingRPC :currently unsupported pending https://github.com/grpc/grpc-go/issues/906. +// Note that using this registration option will cause many gRPC library features to stop working. Consider using RegisterOperatorServiceHandlerFromEndpoint instead. +// GRPC interceptors will not work for this type of registration. To use interceptors, you must use the "runtime.WithMiddlewares" option in the "runtime.NewServeMux" call. +func RegisterOperatorServiceHandlerServer(ctx context.Context, mux *runtime.ServeMux, server OperatorServiceServer) error { + + mux.Handle("GET", pattern_OperatorService_ListSearchAttributes_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/ListSearchAttributes", runtime.WithHTTPPathPattern("/cluster/namespaces/{namespace}/search-attributes")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_OperatorService_ListSearchAttributes_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_ListSearchAttributes_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_OperatorService_ListSearchAttributes_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/ListSearchAttributes", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/search-attributes")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_OperatorService_ListSearchAttributes_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_ListSearchAttributes_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_OperatorService_GetNexusEndpoint_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/GetNexusEndpoint", runtime.WithHTTPPathPattern("/cluster/nexus/endpoints/{id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_OperatorService_GetNexusEndpoint_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_GetNexusEndpoint_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_OperatorService_GetNexusEndpoint_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/GetNexusEndpoint", runtime.WithHTTPPathPattern("/api/v1/nexus/endpoints/{id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_OperatorService_GetNexusEndpoint_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_GetNexusEndpoint_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_OperatorService_CreateNexusEndpoint_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/CreateNexusEndpoint", runtime.WithHTTPPathPattern("/cluster/nexus/endpoints")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_OperatorService_CreateNexusEndpoint_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_CreateNexusEndpoint_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_OperatorService_CreateNexusEndpoint_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/CreateNexusEndpoint", runtime.WithHTTPPathPattern("/api/v1/nexus/endpoints")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_OperatorService_CreateNexusEndpoint_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_CreateNexusEndpoint_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_OperatorService_UpdateNexusEndpoint_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/UpdateNexusEndpoint", runtime.WithHTTPPathPattern("/cluster/nexus/endpoints/{id}/update")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_OperatorService_UpdateNexusEndpoint_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_UpdateNexusEndpoint_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_OperatorService_UpdateNexusEndpoint_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/UpdateNexusEndpoint", runtime.WithHTTPPathPattern("/api/v1/nexus/endpoints/{id}/update")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_OperatorService_UpdateNexusEndpoint_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_UpdateNexusEndpoint_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_OperatorService_DeleteNexusEndpoint_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/DeleteNexusEndpoint", runtime.WithHTTPPathPattern("/cluster/nexus/endpoints/{id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_OperatorService_DeleteNexusEndpoint_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_DeleteNexusEndpoint_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_OperatorService_DeleteNexusEndpoint_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/DeleteNexusEndpoint", runtime.WithHTTPPathPattern("/api/v1/nexus/endpoints/{id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_OperatorService_DeleteNexusEndpoint_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_DeleteNexusEndpoint_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_OperatorService_ListNexusEndpoints_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/ListNexusEndpoints", runtime.WithHTTPPathPattern("/cluster/nexus/endpoints")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_OperatorService_ListNexusEndpoints_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_ListNexusEndpoints_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_OperatorService_ListNexusEndpoints_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/ListNexusEndpoints", runtime.WithHTTPPathPattern("/api/v1/nexus/endpoints")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_OperatorService_ListNexusEndpoints_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_ListNexusEndpoints_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + return nil +} + +// RegisterOperatorServiceHandlerFromEndpoint is same as RegisterOperatorServiceHandler but +// automatically dials to "endpoint" and closes the connection when "ctx" gets done. +func RegisterOperatorServiceHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) { + conn, err := grpc.NewClient(endpoint, opts...) + if err != nil { + return err + } + defer func() { + if err != nil { + if cerr := conn.Close(); cerr != nil { + grpclog.Errorf("Failed to close conn to %s: %v", endpoint, cerr) + } + return + } + go func() { + <-ctx.Done() + if cerr := conn.Close(); cerr != nil { + grpclog.Errorf("Failed to close conn to %s: %v", endpoint, cerr) + } + }() + }() + + return RegisterOperatorServiceHandler(ctx, mux, conn) +} + +// RegisterOperatorServiceHandler registers the http handlers for service OperatorService to "mux". +// The handlers forward requests to the grpc endpoint over "conn". +func RegisterOperatorServiceHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error { + return RegisterOperatorServiceHandlerClient(ctx, mux, NewOperatorServiceClient(conn)) +} + +// RegisterOperatorServiceHandlerClient registers the http handlers for service OperatorService +// to "mux". The handlers forward requests to the grpc endpoint over the given implementation of "OperatorServiceClient". +// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "OperatorServiceClient" +// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in +// "OperatorServiceClient" to call the correct interceptors. This client ignores the HTTP middlewares. +func RegisterOperatorServiceHandlerClient(ctx context.Context, mux *runtime.ServeMux, client OperatorServiceClient) error { + + mux.Handle("GET", pattern_OperatorService_ListSearchAttributes_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/ListSearchAttributes", runtime.WithHTTPPathPattern("/cluster/namespaces/{namespace}/search-attributes")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_OperatorService_ListSearchAttributes_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_ListSearchAttributes_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_OperatorService_ListSearchAttributes_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/ListSearchAttributes", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/search-attributes")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_OperatorService_ListSearchAttributes_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_ListSearchAttributes_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_OperatorService_GetNexusEndpoint_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/GetNexusEndpoint", runtime.WithHTTPPathPattern("/cluster/nexus/endpoints/{id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_OperatorService_GetNexusEndpoint_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_GetNexusEndpoint_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_OperatorService_GetNexusEndpoint_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/GetNexusEndpoint", runtime.WithHTTPPathPattern("/api/v1/nexus/endpoints/{id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_OperatorService_GetNexusEndpoint_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_GetNexusEndpoint_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_OperatorService_CreateNexusEndpoint_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/CreateNexusEndpoint", runtime.WithHTTPPathPattern("/cluster/nexus/endpoints")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_OperatorService_CreateNexusEndpoint_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_CreateNexusEndpoint_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_OperatorService_CreateNexusEndpoint_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/CreateNexusEndpoint", runtime.WithHTTPPathPattern("/api/v1/nexus/endpoints")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_OperatorService_CreateNexusEndpoint_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_CreateNexusEndpoint_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_OperatorService_UpdateNexusEndpoint_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/UpdateNexusEndpoint", runtime.WithHTTPPathPattern("/cluster/nexus/endpoints/{id}/update")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_OperatorService_UpdateNexusEndpoint_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_UpdateNexusEndpoint_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_OperatorService_UpdateNexusEndpoint_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/UpdateNexusEndpoint", runtime.WithHTTPPathPattern("/api/v1/nexus/endpoints/{id}/update")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_OperatorService_UpdateNexusEndpoint_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_UpdateNexusEndpoint_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_OperatorService_DeleteNexusEndpoint_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/DeleteNexusEndpoint", runtime.WithHTTPPathPattern("/cluster/nexus/endpoints/{id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_OperatorService_DeleteNexusEndpoint_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_DeleteNexusEndpoint_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_OperatorService_DeleteNexusEndpoint_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/DeleteNexusEndpoint", runtime.WithHTTPPathPattern("/api/v1/nexus/endpoints/{id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_OperatorService_DeleteNexusEndpoint_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_DeleteNexusEndpoint_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_OperatorService_ListNexusEndpoints_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/ListNexusEndpoints", runtime.WithHTTPPathPattern("/cluster/nexus/endpoints")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_OperatorService_ListNexusEndpoints_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_ListNexusEndpoints_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_OperatorService_ListNexusEndpoints_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.operatorservice.v1.OperatorService/ListNexusEndpoints", runtime.WithHTTPPathPattern("/api/v1/nexus/endpoints")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_OperatorService_ListNexusEndpoints_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_OperatorService_ListNexusEndpoints_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + return nil +} + +var ( + pattern_OperatorService_ListSearchAttributes_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2, 2, 3}, []string{"cluster", "namespaces", "namespace", "search-attributes"}, "")) + + pattern_OperatorService_ListSearchAttributes_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"api", "v1", "namespaces", "namespace", "search-attributes"}, "")) + + pattern_OperatorService_GetNexusEndpoint_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"cluster", "nexus", "endpoints", "id"}, "")) + + pattern_OperatorService_GetNexusEndpoint_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 1, 0, 4, 1, 5, 4}, []string{"api", "v1", "nexus", "endpoints", "id"}, "")) + + pattern_OperatorService_CreateNexusEndpoint_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"cluster", "nexus", "endpoints"}, "")) + + pattern_OperatorService_CreateNexusEndpoint_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"api", "v1", "nexus", "endpoints"}, "")) + + pattern_OperatorService_UpdateNexusEndpoint_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"cluster", "nexus", "endpoints", "id", "update"}, "")) + + pattern_OperatorService_UpdateNexusEndpoint_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 1, 0, 4, 1, 5, 4, 2, 5}, []string{"api", "v1", "nexus", "endpoints", "id", "update"}, "")) + + pattern_OperatorService_DeleteNexusEndpoint_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"cluster", "nexus", "endpoints", "id"}, "")) + + pattern_OperatorService_DeleteNexusEndpoint_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 1, 0, 4, 1, 5, 4}, []string{"api", "v1", "nexus", "endpoints", "id"}, "")) + + pattern_OperatorService_ListNexusEndpoints_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"cluster", "nexus", "endpoints"}, "")) + + pattern_OperatorService_ListNexusEndpoints_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"api", "v1", "nexus", "endpoints"}, "")) +) + +var ( + forward_OperatorService_ListSearchAttributes_0 = runtime.ForwardResponseMessage + + forward_OperatorService_ListSearchAttributes_1 = runtime.ForwardResponseMessage + + forward_OperatorService_GetNexusEndpoint_0 = runtime.ForwardResponseMessage + + forward_OperatorService_GetNexusEndpoint_1 = runtime.ForwardResponseMessage + + forward_OperatorService_CreateNexusEndpoint_0 = runtime.ForwardResponseMessage + + forward_OperatorService_CreateNexusEndpoint_1 = runtime.ForwardResponseMessage + + forward_OperatorService_UpdateNexusEndpoint_0 = runtime.ForwardResponseMessage + + forward_OperatorService_UpdateNexusEndpoint_1 = runtime.ForwardResponseMessage + + forward_OperatorService_DeleteNexusEndpoint_0 = runtime.ForwardResponseMessage + + forward_OperatorService_DeleteNexusEndpoint_1 = runtime.ForwardResponseMessage + + forward_OperatorService_ListNexusEndpoints_0 = runtime.ForwardResponseMessage + + forward_OperatorService_ListNexusEndpoints_1 = runtime.ForwardResponseMessage +) diff --git a/vendor/go.temporal.io/api/operatorservice/v1/service_grpc.pb.go b/vendor/go.temporal.io/api/operatorservice/v1/service_grpc.pb.go new file mode 100644 index 00000000000..c77088c6bb2 --- /dev/null +++ b/vendor/go.temporal.io/api/operatorservice/v1/service_grpc.pb.go @@ -0,0 +1,624 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// plugins: +// - protoc-gen-go-grpc +// - protoc +// source: temporal/api/operatorservice/v1/service.proto + +package operatorservice + +import ( + context "context" + + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.64.0 or later. +const _ = grpc.SupportPackageIsVersion9 + +const ( + OperatorService_AddSearchAttributes_FullMethodName = "/temporal.api.operatorservice.v1.OperatorService/AddSearchAttributes" + OperatorService_RemoveSearchAttributes_FullMethodName = "/temporal.api.operatorservice.v1.OperatorService/RemoveSearchAttributes" + OperatorService_ListSearchAttributes_FullMethodName = "/temporal.api.operatorservice.v1.OperatorService/ListSearchAttributes" + OperatorService_DeleteNamespace_FullMethodName = "/temporal.api.operatorservice.v1.OperatorService/DeleteNamespace" + OperatorService_AddOrUpdateRemoteCluster_FullMethodName = "/temporal.api.operatorservice.v1.OperatorService/AddOrUpdateRemoteCluster" + OperatorService_RemoveRemoteCluster_FullMethodName = "/temporal.api.operatorservice.v1.OperatorService/RemoveRemoteCluster" + OperatorService_ListClusters_FullMethodName = "/temporal.api.operatorservice.v1.OperatorService/ListClusters" + OperatorService_GetNexusEndpoint_FullMethodName = "/temporal.api.operatorservice.v1.OperatorService/GetNexusEndpoint" + OperatorService_CreateNexusEndpoint_FullMethodName = "/temporal.api.operatorservice.v1.OperatorService/CreateNexusEndpoint" + OperatorService_UpdateNexusEndpoint_FullMethodName = "/temporal.api.operatorservice.v1.OperatorService/UpdateNexusEndpoint" + OperatorService_DeleteNexusEndpoint_FullMethodName = "/temporal.api.operatorservice.v1.OperatorService/DeleteNexusEndpoint" + OperatorService_ListNexusEndpoints_FullMethodName = "/temporal.api.operatorservice.v1.OperatorService/ListNexusEndpoints" +) + +// OperatorServiceClient is the client API for OperatorService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// OperatorService API defines how Temporal SDKs and other clients interact with the Temporal server +// to perform administrative functions like registering a search attribute or a namespace. +// APIs in this file could be not compatible with Temporal Cloud, hence it's usage in SDKs should be limited by +// designated APIs that clearly state that they shouldn't be used by the main Application (Workflows & Activities) framework. +type OperatorServiceClient interface { + // AddSearchAttributes add custom search attributes. + // + // Returns ALREADY_EXISTS status code if a Search Attribute with any of the specified names already exists + // Returns INTERNAL status code with temporal.api.errordetails.v1.SystemWorkflowFailure in Error Details if registration process fails, + AddSearchAttributes(ctx context.Context, in *AddSearchAttributesRequest, opts ...grpc.CallOption) (*AddSearchAttributesResponse, error) + // RemoveSearchAttributes removes custom search attributes. + // + // Returns NOT_FOUND status code if a Search Attribute with any of the specified names is not registered + RemoveSearchAttributes(ctx context.Context, in *RemoveSearchAttributesRequest, opts ...grpc.CallOption) (*RemoveSearchAttributesResponse, error) + // ListSearchAttributes returns comprehensive information about search attributes. + ListSearchAttributes(ctx context.Context, in *ListSearchAttributesRequest, opts ...grpc.CallOption) (*ListSearchAttributesResponse, error) + // DeleteNamespace synchronously deletes a namespace and asynchronously reclaims all namespace resources. + DeleteNamespace(ctx context.Context, in *DeleteNamespaceRequest, opts ...grpc.CallOption) (*DeleteNamespaceResponse, error) + // AddOrUpdateRemoteCluster adds or updates remote cluster. + AddOrUpdateRemoteCluster(ctx context.Context, in *AddOrUpdateRemoteClusterRequest, opts ...grpc.CallOption) (*AddOrUpdateRemoteClusterResponse, error) + // RemoveRemoteCluster removes remote cluster. + RemoveRemoteCluster(ctx context.Context, in *RemoveRemoteClusterRequest, opts ...grpc.CallOption) (*RemoveRemoteClusterResponse, error) + // ListClusters returns information about Temporal clusters. + ListClusters(ctx context.Context, in *ListClustersRequest, opts ...grpc.CallOption) (*ListClustersResponse, error) + // Get a registered Nexus endpoint by ID. The returned version can be used for optimistic updates. + GetNexusEndpoint(ctx context.Context, in *GetNexusEndpointRequest, opts ...grpc.CallOption) (*GetNexusEndpointResponse, error) + // Create a Nexus endpoint. This will fail if an endpoint with the same name is already registered with a status of + // ALREADY_EXISTS. + // Returns the created endpoint with its initial version. You may use this version for subsequent updates. + CreateNexusEndpoint(ctx context.Context, in *CreateNexusEndpointRequest, opts ...grpc.CallOption) (*CreateNexusEndpointResponse, error) + // Optimistically update a Nexus endpoint based on provided version as obtained via the `GetNexusEndpoint` or + // `ListNexusEndpointResponse` APIs. This will fail with a status of FAILED_PRECONDITION if the version does not + // match. + // Returns the updated endpoint with its updated version. You may use this version for subsequent updates. You don't + // need to increment the version yourself. The server will increment the version for you after each update. + UpdateNexusEndpoint(ctx context.Context, in *UpdateNexusEndpointRequest, opts ...grpc.CallOption) (*UpdateNexusEndpointResponse, error) + // Delete an incoming Nexus service by ID. + DeleteNexusEndpoint(ctx context.Context, in *DeleteNexusEndpointRequest, opts ...grpc.CallOption) (*DeleteNexusEndpointResponse, error) + // List all Nexus endpoints for the cluster, sorted by ID in ascending order. Set page_token in the request to the + // next_page_token field of the previous response to get the next page of results. An empty next_page_token + // indicates that there are no more results. During pagination, a newly added service with an ID lexicographically + // earlier than the previous page's last endpoint's ID may be missed. + ListNexusEndpoints(ctx context.Context, in *ListNexusEndpointsRequest, opts ...grpc.CallOption) (*ListNexusEndpointsResponse, error) +} + +type operatorServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewOperatorServiceClient(cc grpc.ClientConnInterface) OperatorServiceClient { + return &operatorServiceClient{cc} +} + +func (c *operatorServiceClient) AddSearchAttributes(ctx context.Context, in *AddSearchAttributesRequest, opts ...grpc.CallOption) (*AddSearchAttributesResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(AddSearchAttributesResponse) + err := c.cc.Invoke(ctx, OperatorService_AddSearchAttributes_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *operatorServiceClient) RemoveSearchAttributes(ctx context.Context, in *RemoveSearchAttributesRequest, opts ...grpc.CallOption) (*RemoveSearchAttributesResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RemoveSearchAttributesResponse) + err := c.cc.Invoke(ctx, OperatorService_RemoveSearchAttributes_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *operatorServiceClient) ListSearchAttributes(ctx context.Context, in *ListSearchAttributesRequest, opts ...grpc.CallOption) (*ListSearchAttributesResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ListSearchAttributesResponse) + err := c.cc.Invoke(ctx, OperatorService_ListSearchAttributes_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *operatorServiceClient) DeleteNamespace(ctx context.Context, in *DeleteNamespaceRequest, opts ...grpc.CallOption) (*DeleteNamespaceResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DeleteNamespaceResponse) + err := c.cc.Invoke(ctx, OperatorService_DeleteNamespace_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *operatorServiceClient) AddOrUpdateRemoteCluster(ctx context.Context, in *AddOrUpdateRemoteClusterRequest, opts ...grpc.CallOption) (*AddOrUpdateRemoteClusterResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(AddOrUpdateRemoteClusterResponse) + err := c.cc.Invoke(ctx, OperatorService_AddOrUpdateRemoteCluster_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *operatorServiceClient) RemoveRemoteCluster(ctx context.Context, in *RemoveRemoteClusterRequest, opts ...grpc.CallOption) (*RemoveRemoteClusterResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RemoveRemoteClusterResponse) + err := c.cc.Invoke(ctx, OperatorService_RemoveRemoteCluster_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *operatorServiceClient) ListClusters(ctx context.Context, in *ListClustersRequest, opts ...grpc.CallOption) (*ListClustersResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ListClustersResponse) + err := c.cc.Invoke(ctx, OperatorService_ListClusters_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *operatorServiceClient) GetNexusEndpoint(ctx context.Context, in *GetNexusEndpointRequest, opts ...grpc.CallOption) (*GetNexusEndpointResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetNexusEndpointResponse) + err := c.cc.Invoke(ctx, OperatorService_GetNexusEndpoint_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *operatorServiceClient) CreateNexusEndpoint(ctx context.Context, in *CreateNexusEndpointRequest, opts ...grpc.CallOption) (*CreateNexusEndpointResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(CreateNexusEndpointResponse) + err := c.cc.Invoke(ctx, OperatorService_CreateNexusEndpoint_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *operatorServiceClient) UpdateNexusEndpoint(ctx context.Context, in *UpdateNexusEndpointRequest, opts ...grpc.CallOption) (*UpdateNexusEndpointResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(UpdateNexusEndpointResponse) + err := c.cc.Invoke(ctx, OperatorService_UpdateNexusEndpoint_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *operatorServiceClient) DeleteNexusEndpoint(ctx context.Context, in *DeleteNexusEndpointRequest, opts ...grpc.CallOption) (*DeleteNexusEndpointResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DeleteNexusEndpointResponse) + err := c.cc.Invoke(ctx, OperatorService_DeleteNexusEndpoint_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *operatorServiceClient) ListNexusEndpoints(ctx context.Context, in *ListNexusEndpointsRequest, opts ...grpc.CallOption) (*ListNexusEndpointsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ListNexusEndpointsResponse) + err := c.cc.Invoke(ctx, OperatorService_ListNexusEndpoints_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +// OperatorServiceServer is the server API for OperatorService service. +// All implementations must embed UnimplementedOperatorServiceServer +// for forward compatibility. +// +// OperatorService API defines how Temporal SDKs and other clients interact with the Temporal server +// to perform administrative functions like registering a search attribute or a namespace. +// APIs in this file could be not compatible with Temporal Cloud, hence it's usage in SDKs should be limited by +// designated APIs that clearly state that they shouldn't be used by the main Application (Workflows & Activities) framework. +type OperatorServiceServer interface { + // AddSearchAttributes add custom search attributes. + // + // Returns ALREADY_EXISTS status code if a Search Attribute with any of the specified names already exists + // Returns INTERNAL status code with temporal.api.errordetails.v1.SystemWorkflowFailure in Error Details if registration process fails, + AddSearchAttributes(context.Context, *AddSearchAttributesRequest) (*AddSearchAttributesResponse, error) + // RemoveSearchAttributes removes custom search attributes. + // + // Returns NOT_FOUND status code if a Search Attribute with any of the specified names is not registered + RemoveSearchAttributes(context.Context, *RemoveSearchAttributesRequest) (*RemoveSearchAttributesResponse, error) + // ListSearchAttributes returns comprehensive information about search attributes. + ListSearchAttributes(context.Context, *ListSearchAttributesRequest) (*ListSearchAttributesResponse, error) + // DeleteNamespace synchronously deletes a namespace and asynchronously reclaims all namespace resources. + DeleteNamespace(context.Context, *DeleteNamespaceRequest) (*DeleteNamespaceResponse, error) + // AddOrUpdateRemoteCluster adds or updates remote cluster. + AddOrUpdateRemoteCluster(context.Context, *AddOrUpdateRemoteClusterRequest) (*AddOrUpdateRemoteClusterResponse, error) + // RemoveRemoteCluster removes remote cluster. + RemoveRemoteCluster(context.Context, *RemoveRemoteClusterRequest) (*RemoveRemoteClusterResponse, error) + // ListClusters returns information about Temporal clusters. + ListClusters(context.Context, *ListClustersRequest) (*ListClustersResponse, error) + // Get a registered Nexus endpoint by ID. The returned version can be used for optimistic updates. + GetNexusEndpoint(context.Context, *GetNexusEndpointRequest) (*GetNexusEndpointResponse, error) + // Create a Nexus endpoint. This will fail if an endpoint with the same name is already registered with a status of + // ALREADY_EXISTS. + // Returns the created endpoint with its initial version. You may use this version for subsequent updates. + CreateNexusEndpoint(context.Context, *CreateNexusEndpointRequest) (*CreateNexusEndpointResponse, error) + // Optimistically update a Nexus endpoint based on provided version as obtained via the `GetNexusEndpoint` or + // `ListNexusEndpointResponse` APIs. This will fail with a status of FAILED_PRECONDITION if the version does not + // match. + // Returns the updated endpoint with its updated version. You may use this version for subsequent updates. You don't + // need to increment the version yourself. The server will increment the version for you after each update. + UpdateNexusEndpoint(context.Context, *UpdateNexusEndpointRequest) (*UpdateNexusEndpointResponse, error) + // Delete an incoming Nexus service by ID. + DeleteNexusEndpoint(context.Context, *DeleteNexusEndpointRequest) (*DeleteNexusEndpointResponse, error) + // List all Nexus endpoints for the cluster, sorted by ID in ascending order. Set page_token in the request to the + // next_page_token field of the previous response to get the next page of results. An empty next_page_token + // indicates that there are no more results. During pagination, a newly added service with an ID lexicographically + // earlier than the previous page's last endpoint's ID may be missed. + ListNexusEndpoints(context.Context, *ListNexusEndpointsRequest) (*ListNexusEndpointsResponse, error) + mustEmbedUnimplementedOperatorServiceServer() +} + +// UnimplementedOperatorServiceServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedOperatorServiceServer struct{} + +func (UnimplementedOperatorServiceServer) AddSearchAttributes(context.Context, *AddSearchAttributesRequest) (*AddSearchAttributesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AddSearchAttributes not implemented") +} +func (UnimplementedOperatorServiceServer) RemoveSearchAttributes(context.Context, *RemoveSearchAttributesRequest) (*RemoveSearchAttributesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RemoveSearchAttributes not implemented") +} +func (UnimplementedOperatorServiceServer) ListSearchAttributes(context.Context, *ListSearchAttributesRequest) (*ListSearchAttributesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListSearchAttributes not implemented") +} +func (UnimplementedOperatorServiceServer) DeleteNamespace(context.Context, *DeleteNamespaceRequest) (*DeleteNamespaceResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteNamespace not implemented") +} +func (UnimplementedOperatorServiceServer) AddOrUpdateRemoteCluster(context.Context, *AddOrUpdateRemoteClusterRequest) (*AddOrUpdateRemoteClusterResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AddOrUpdateRemoteCluster not implemented") +} +func (UnimplementedOperatorServiceServer) RemoveRemoteCluster(context.Context, *RemoveRemoteClusterRequest) (*RemoveRemoteClusterResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RemoveRemoteCluster not implemented") +} +func (UnimplementedOperatorServiceServer) ListClusters(context.Context, *ListClustersRequest) (*ListClustersResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListClusters not implemented") +} +func (UnimplementedOperatorServiceServer) GetNexusEndpoint(context.Context, *GetNexusEndpointRequest) (*GetNexusEndpointResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetNexusEndpoint not implemented") +} +func (UnimplementedOperatorServiceServer) CreateNexusEndpoint(context.Context, *CreateNexusEndpointRequest) (*CreateNexusEndpointResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateNexusEndpoint not implemented") +} +func (UnimplementedOperatorServiceServer) UpdateNexusEndpoint(context.Context, *UpdateNexusEndpointRequest) (*UpdateNexusEndpointResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateNexusEndpoint not implemented") +} +func (UnimplementedOperatorServiceServer) DeleteNexusEndpoint(context.Context, *DeleteNexusEndpointRequest) (*DeleteNexusEndpointResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteNexusEndpoint not implemented") +} +func (UnimplementedOperatorServiceServer) ListNexusEndpoints(context.Context, *ListNexusEndpointsRequest) (*ListNexusEndpointsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListNexusEndpoints not implemented") +} +func (UnimplementedOperatorServiceServer) mustEmbedUnimplementedOperatorServiceServer() {} +func (UnimplementedOperatorServiceServer) testEmbeddedByValue() {} + +// UnsafeOperatorServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to OperatorServiceServer will +// result in compilation errors. +type UnsafeOperatorServiceServer interface { + mustEmbedUnimplementedOperatorServiceServer() +} + +func RegisterOperatorServiceServer(s grpc.ServiceRegistrar, srv OperatorServiceServer) { + // If the following call pancis, it indicates UnimplementedOperatorServiceServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } + s.RegisterService(&OperatorService_ServiceDesc, srv) +} + +func _OperatorService_AddSearchAttributes_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AddSearchAttributesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(OperatorServiceServer).AddSearchAttributes(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: OperatorService_AddSearchAttributes_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(OperatorServiceServer).AddSearchAttributes(ctx, req.(*AddSearchAttributesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _OperatorService_RemoveSearchAttributes_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RemoveSearchAttributesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(OperatorServiceServer).RemoveSearchAttributes(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: OperatorService_RemoveSearchAttributes_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(OperatorServiceServer).RemoveSearchAttributes(ctx, req.(*RemoveSearchAttributesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _OperatorService_ListSearchAttributes_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListSearchAttributesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(OperatorServiceServer).ListSearchAttributes(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: OperatorService_ListSearchAttributes_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(OperatorServiceServer).ListSearchAttributes(ctx, req.(*ListSearchAttributesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _OperatorService_DeleteNamespace_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteNamespaceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(OperatorServiceServer).DeleteNamespace(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: OperatorService_DeleteNamespace_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(OperatorServiceServer).DeleteNamespace(ctx, req.(*DeleteNamespaceRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _OperatorService_AddOrUpdateRemoteCluster_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AddOrUpdateRemoteClusterRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(OperatorServiceServer).AddOrUpdateRemoteCluster(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: OperatorService_AddOrUpdateRemoteCluster_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(OperatorServiceServer).AddOrUpdateRemoteCluster(ctx, req.(*AddOrUpdateRemoteClusterRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _OperatorService_RemoveRemoteCluster_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RemoveRemoteClusterRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(OperatorServiceServer).RemoveRemoteCluster(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: OperatorService_RemoveRemoteCluster_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(OperatorServiceServer).RemoveRemoteCluster(ctx, req.(*RemoveRemoteClusterRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _OperatorService_ListClusters_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListClustersRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(OperatorServiceServer).ListClusters(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: OperatorService_ListClusters_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(OperatorServiceServer).ListClusters(ctx, req.(*ListClustersRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _OperatorService_GetNexusEndpoint_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetNexusEndpointRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(OperatorServiceServer).GetNexusEndpoint(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: OperatorService_GetNexusEndpoint_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(OperatorServiceServer).GetNexusEndpoint(ctx, req.(*GetNexusEndpointRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _OperatorService_CreateNexusEndpoint_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateNexusEndpointRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(OperatorServiceServer).CreateNexusEndpoint(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: OperatorService_CreateNexusEndpoint_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(OperatorServiceServer).CreateNexusEndpoint(ctx, req.(*CreateNexusEndpointRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _OperatorService_UpdateNexusEndpoint_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateNexusEndpointRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(OperatorServiceServer).UpdateNexusEndpoint(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: OperatorService_UpdateNexusEndpoint_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(OperatorServiceServer).UpdateNexusEndpoint(ctx, req.(*UpdateNexusEndpointRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _OperatorService_DeleteNexusEndpoint_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteNexusEndpointRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(OperatorServiceServer).DeleteNexusEndpoint(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: OperatorService_DeleteNexusEndpoint_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(OperatorServiceServer).DeleteNexusEndpoint(ctx, req.(*DeleteNexusEndpointRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _OperatorService_ListNexusEndpoints_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListNexusEndpointsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(OperatorServiceServer).ListNexusEndpoints(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: OperatorService_ListNexusEndpoints_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(OperatorServiceServer).ListNexusEndpoints(ctx, req.(*ListNexusEndpointsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// OperatorService_ServiceDesc is the grpc.ServiceDesc for OperatorService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var OperatorService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "temporal.api.operatorservice.v1.OperatorService", + HandlerType: (*OperatorServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "AddSearchAttributes", + Handler: _OperatorService_AddSearchAttributes_Handler, + }, + { + MethodName: "RemoveSearchAttributes", + Handler: _OperatorService_RemoveSearchAttributes_Handler, + }, + { + MethodName: "ListSearchAttributes", + Handler: _OperatorService_ListSearchAttributes_Handler, + }, + { + MethodName: "DeleteNamespace", + Handler: _OperatorService_DeleteNamespace_Handler, + }, + { + MethodName: "AddOrUpdateRemoteCluster", + Handler: _OperatorService_AddOrUpdateRemoteCluster_Handler, + }, + { + MethodName: "RemoveRemoteCluster", + Handler: _OperatorService_RemoveRemoteCluster_Handler, + }, + { + MethodName: "ListClusters", + Handler: _OperatorService_ListClusters_Handler, + }, + { + MethodName: "GetNexusEndpoint", + Handler: _OperatorService_GetNexusEndpoint_Handler, + }, + { + MethodName: "CreateNexusEndpoint", + Handler: _OperatorService_CreateNexusEndpoint_Handler, + }, + { + MethodName: "UpdateNexusEndpoint", + Handler: _OperatorService_UpdateNexusEndpoint_Handler, + }, + { + MethodName: "DeleteNexusEndpoint", + Handler: _OperatorService_DeleteNexusEndpoint_Handler, + }, + { + MethodName: "ListNexusEndpoints", + Handler: _OperatorService_ListNexusEndpoints_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "temporal/api/operatorservice/v1/service.proto", +} diff --git a/vendor/go.temporal.io/api/protocol/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/protocol/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..02b0334b555 --- /dev/null +++ b/vendor/go.temporal.io/api/protocol/v1/message.go-helpers.pb.go @@ -0,0 +1,65 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package protocol + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type Message to the protobuf v3 wire format +func (val *Message) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Message from the protobuf v3 wire format +func (val *Message) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Message) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Message values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Message) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Message + switch t := that.(type) { + case *Message: + that1 = t + case Message: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/protocol/v1/message.pb.go b/vendor/go.temporal.io/api/protocol/v1/message.pb.go new file mode 100644 index 00000000000..1df9f779af7 --- /dev/null +++ b/vendor/go.temporal.io/api/protocol/v1/message.pb.go @@ -0,0 +1,265 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/protocol/v1/message.proto + +package protocol + +import ( + reflect "reflect" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + anypb "google.golang.org/protobuf/types/known/anypb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// (-- api-linter: core::0146::any=disabled +// +// aip.dev/not-precedent: We want runtime extensibility for the body field --) +type Message struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // An ID for this specific message. + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + // Identifies the specific instance of a protocol to which this message + // belongs. + ProtocolInstanceId string `protobuf:"bytes,2,opt,name=protocol_instance_id,json=protocolInstanceId,proto3" json:"protocol_instance_id,omitempty"` + // The event ID or command ID after which this message can be delivered. The + // effects of history up to and including this event ID should be visible to + // the code that handles this message. Omit to opt out of sequencing. + // + // Types that are assignable to SequencingId: + // + // *Message_EventId + // *Message_CommandIndex + SequencingId isMessage_SequencingId `protobuf_oneof:"sequencing_id"` + // The opaque data carried by this message. The protocol type can be + // extracted from the package name of the message carried inside the Any. + Body *anypb.Any `protobuf:"bytes,5,opt,name=body,proto3" json:"body,omitempty"` +} + +func (x *Message) Reset() { + *x = Message{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_protocol_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Message) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Message) ProtoMessage() {} + +func (x *Message) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_protocol_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Message.ProtoReflect.Descriptor instead. +func (*Message) Descriptor() ([]byte, []int) { + return file_temporal_api_protocol_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *Message) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *Message) GetProtocolInstanceId() string { + if x != nil { + return x.ProtocolInstanceId + } + return "" +} + +func (m *Message) GetSequencingId() isMessage_SequencingId { + if m != nil { + return m.SequencingId + } + return nil +} + +func (x *Message) GetEventId() int64 { + if x, ok := x.GetSequencingId().(*Message_EventId); ok { + return x.EventId + } + return 0 +} + +func (x *Message) GetCommandIndex() int64 { + if x, ok := x.GetSequencingId().(*Message_CommandIndex); ok { + return x.CommandIndex + } + return 0 +} + +func (x *Message) GetBody() *anypb.Any { + if x != nil { + return x.Body + } + return nil +} + +type isMessage_SequencingId interface { + isMessage_SequencingId() +} + +type Message_EventId struct { + EventId int64 `protobuf:"varint,3,opt,name=event_id,json=eventId,proto3,oneof"` +} + +type Message_CommandIndex struct { + CommandIndex int64 `protobuf:"varint,4,opt,name=command_index,json=commandIndex,proto3,oneof"` +} + +func (*Message_EventId) isMessage_SequencingId() {} + +func (*Message_CommandIndex) isMessage_SequencingId() {} + +var File_temporal_api_protocol_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_protocol_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x26, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x18, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x76, 0x31, 0x1a, + 0x19, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, + 0x61, 0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xde, 0x01, 0x0a, 0x07, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x12, 0x12, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x02, 0x69, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, 0x14, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x63, 0x6f, 0x6c, 0x5f, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x49, 0x6e, 0x73, + 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1f, 0x0a, 0x08, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x07, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, 0x0a, 0x0d, 0x63, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x03, 0x48, 0x00, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x42, 0x02, 0x68, 0x00, + 0x42, 0x0f, 0x0a, 0x0d, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x64, + 0x42, 0x93, 0x01, 0x0a, 0x1b, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x76, 0x31, 0x42, + 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x27, + 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2f, 0x76, 0x31, 0x3b, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0xaa, 0x02, 0x1a, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x56, + 0x31, 0xea, 0x02, 0x1d, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, + 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x3a, 0x56, 0x31, + 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_protocol_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_protocol_v1_message_proto_rawDescData = file_temporal_api_protocol_v1_message_proto_rawDesc +) + +func file_temporal_api_protocol_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_protocol_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_protocol_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_protocol_v1_message_proto_rawDescData) + }) + return file_temporal_api_protocol_v1_message_proto_rawDescData +} + +var file_temporal_api_protocol_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 1) +var file_temporal_api_protocol_v1_message_proto_goTypes = []any{ + (*Message)(nil), // 0: temporal.api.protocol.v1.Message + (*anypb.Any)(nil), // 1: google.protobuf.Any +} +var file_temporal_api_protocol_v1_message_proto_depIdxs = []int32{ + 1, // 0: temporal.api.protocol.v1.Message.body:type_name -> google.protobuf.Any + 1, // [1:1] is the sub-list for method output_type + 1, // [1:1] is the sub-list for method input_type + 1, // [1:1] is the sub-list for extension type_name + 1, // [1:1] is the sub-list for extension extendee + 0, // [0:1] is the sub-list for field type_name +} + +func init() { file_temporal_api_protocol_v1_message_proto_init() } +func file_temporal_api_protocol_v1_message_proto_init() { + if File_temporal_api_protocol_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_protocol_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*Message); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_temporal_api_protocol_v1_message_proto_msgTypes[0].OneofWrappers = []any{ + (*Message_EventId)(nil), + (*Message_CommandIndex)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_protocol_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 1, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_protocol_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_protocol_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_protocol_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_protocol_v1_message_proto = out.File + file_temporal_api_protocol_v1_message_proto_rawDesc = nil + file_temporal_api_protocol_v1_message_proto_goTypes = nil + file_temporal_api_protocol_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/proxy/interceptor.go b/vendor/go.temporal.io/api/proxy/interceptor.go new file mode 100644 index 00000000000..9ff57ea8a9f --- /dev/null +++ b/vendor/go.temporal.io/api/proxy/interceptor.go @@ -0,0 +1,2918 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by proxygenerator; DO NOT EDIT. + +package proxy + +import ( + "context" + "fmt" + + "go.temporal.io/api/batch/v1" + "go.temporal.io/api/command/v1" + "go.temporal.io/api/common/v1" + "go.temporal.io/api/export/v1" + "go.temporal.io/api/failure/v1" + "go.temporal.io/api/history/v1" + "go.temporal.io/api/nexus/v1" + "go.temporal.io/api/query/v1" + "go.temporal.io/api/schedule/v1" + "go.temporal.io/api/sdk/v1" + "go.temporal.io/api/update/v1" + "go.temporal.io/api/workflow/v1" + "go.temporal.io/api/workflowservice/v1" + "google.golang.org/grpc" + "google.golang.org/protobuf/proto" +) + +// VisitPayloadsContext provides Payload context for visitor functions. +type VisitPayloadsContext struct { + context.Context + // The parent message for this payload. + Parent proto.Message + // If true, a single payload is given and a single payload must be returned. + SinglePayloadRequired bool +} + +// VisitPayloadsOptions configure visitor behaviour. +type VisitPayloadsOptions struct { + // Context is the same for every call of a visit, callers should not store it. This must never + // return an empty set of payloads. + Visitor func(*VisitPayloadsContext, []*common.Payload) ([]*common.Payload, error) + // Don't visit search attribute payloads. + SkipSearchAttributes bool +} + +// VisitPayloads calls the options.Visitor function for every Payload proto within msg. +func VisitPayloads(ctx context.Context, msg proto.Message, options VisitPayloadsOptions) error { + visitCtx := VisitPayloadsContext{Context: ctx, Parent: msg} + + return visitPayloads(&visitCtx, &options, msg) +} + +// PayloadVisitorInterceptorOptions configures outbound/inbound interception of Payloads within msgs. +type PayloadVisitorInterceptorOptions struct { + // Visit options for outbound messages + Outbound *VisitPayloadsOptions + // Visit options for inbound messages + Inbound *VisitPayloadsOptions +} + +// NewPayloadVisitorInterceptor creates a new gRPC interceptor for workflowservice messages. +func NewPayloadVisitorInterceptor(options PayloadVisitorInterceptorOptions) (grpc.UnaryClientInterceptor, error) { + return func(ctx context.Context, method string, req, response interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + if reqMsg, ok := req.(proto.Message); ok && options.Outbound != nil { + err := VisitPayloads(ctx, reqMsg, *options.Outbound) + if err != nil { + return err + } + } + + err := invoker(ctx, method, req, response, cc, opts...) + if err != nil { + return err + } + + if resMsg, ok := response.(proto.Message); ok && options.Inbound != nil { + return VisitPayloads(ctx, resMsg, *options.Inbound) + } + + return nil + }, nil +} + +// VisitFailuresContext provides Failure context for visitor functions. +type VisitFailuresContext struct { + context.Context + // The parent message for this failure. + Parent proto.Message +} + +// VisitFailuresOptions configure visitor behaviour. +type VisitFailuresOptions struct { + // Context is the same for every call of a visit, callers should not store it. + // Visitor is free to mutate the passed failure struct. + Visitor func(*VisitFailuresContext, *failure.Failure) error +} + +// VisitFailures calls the options.Visitor function for every Failure proto within msg. +func VisitFailures(ctx context.Context, msg proto.Message, options VisitFailuresOptions) error { + visitCtx := VisitFailuresContext{Context: ctx, Parent: msg} + + return visitFailures(&visitCtx, &options, msg) +} + +// FailureVisitorInterceptorOptions configures outbound/inbound interception of Failures within msgs. +type FailureVisitorInterceptorOptions struct { + // Visit options for outbound messages + Outbound *VisitFailuresOptions + // Visit options for inbound messages + Inbound *VisitFailuresOptions +} + +// NewFailureVisitorInterceptor creates a new gRPC interceptor for workflowservice messages. +func NewFailureVisitorInterceptor(options FailureVisitorInterceptorOptions) (grpc.UnaryClientInterceptor, error) { + return func(ctx context.Context, method string, req, response interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + if reqMsg, ok := req.(proto.Message); ok && options.Outbound != nil { + err := VisitFailures(ctx, reqMsg, *options.Outbound) + if err != nil { + return err + } + } + + err := invoker(ctx, method, req, response, cc, opts...) + if err != nil { + return err + } + + if resMsg, ok := response.(proto.Message); ok && options.Inbound != nil { + return VisitFailures(ctx, resMsg, *options.Inbound) + } + + return nil + }, nil +} + +func visitPayload(ctx *VisitPayloadsContext, options *VisitPayloadsOptions, msg *common.Payload) (*common.Payload, error) { + ctx.SinglePayloadRequired = true + + newPayloads, err := options.Visitor(ctx, []*common.Payload{msg}) + if err != nil { + return nil, err + } + + if len(newPayloads) != 1 { + return nil, fmt.Errorf("visitor func must return 1 payload when SinglePayloadRequired = true") + } + + return newPayloads[0], nil +} + +func visitPayloads(ctx *VisitPayloadsContext, options *VisitPayloadsOptions, objs ...interface{}) error { + for i, obj := range objs { + ctx.SinglePayloadRequired = false + + switch o := obj.(type) { + case *common.Payload: + if o == nil { + continue + } + no, err := visitPayload(ctx, options, o) + if err != nil { + return err + } + objs[i] = no + case map[string]*common.Payload: + for ix, x := range o { + if nx, err := visitPayload(ctx, options, x); err != nil { + return err + } else { + o[ix] = nx + } + } + case *common.Payloads: + if o == nil { + continue + } + newPayloads, err := options.Visitor(ctx, o.Payloads) + if err != nil { + return err + } + o.Payloads = newPayloads + case map[string]*common.Payloads: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *batch.BatchOperationSignal: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + o.GetInput(), + ); err != nil { + return err + } + + case *batch.BatchOperationTermination: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + ); err != nil { + return err + } + + case *command.CancelWorkflowExecutionCommandAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + ); err != nil { + return err + } + + case []*command.Command: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *command.Command: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetCancelWorkflowExecutionCommandAttributes(), + o.GetCompleteWorkflowExecutionCommandAttributes(), + o.GetContinueAsNewWorkflowExecutionCommandAttributes(), + o.GetFailWorkflowExecutionCommandAttributes(), + o.GetModifyWorkflowPropertiesCommandAttributes(), + o.GetRecordMarkerCommandAttributes(), + o.GetScheduleActivityTaskCommandAttributes(), + o.GetScheduleNexusOperationCommandAttributes(), + o.GetSignalExternalWorkflowExecutionCommandAttributes(), + o.GetStartChildWorkflowExecutionCommandAttributes(), + o.GetUpsertWorkflowSearchAttributesCommandAttributes(), + o.GetUserMetadata(), + ); err != nil { + return err + } + + case *command.CompleteWorkflowExecutionCommandAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetResult(), + ); err != nil { + return err + } + + case *command.ContinueAsNewWorkflowExecutionCommandAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + o.GetHeader(), + o.GetInput(), + o.GetLastCompletionResult(), + o.GetMemo(), + o.GetSearchAttributes(), + ); err != nil { + return err + } + + case *command.FailWorkflowExecutionCommandAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *command.ModifyWorkflowPropertiesCommandAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetUpsertedMemo(), + ); err != nil { + return err + } + + case *command.RecordMarkerCommandAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + o.GetFailure(), + o.GetHeader(), + ); err != nil { + return err + } + + case *command.ScheduleActivityTaskCommandAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + o.GetInput(), + ); err != nil { + return err + } + + case *command.ScheduleNexusOperationCommandAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetInput(), + ); err != nil { + return err + } + + case *command.SignalExternalWorkflowExecutionCommandAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + o.GetInput(), + ); err != nil { + return err + } + + case *command.StartChildWorkflowExecutionCommandAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + o.GetInput(), + o.GetMemo(), + o.GetSearchAttributes(), + ); err != nil { + return err + } + + case *command.UpsertWorkflowSearchAttributesCommandAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetSearchAttributes(), + ); err != nil { + return err + } + + case *common.Header: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFields(), + ); err != nil { + return err + } + + case *common.Memo: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFields(), + ); err != nil { + return err + } + + case *common.SearchAttributes: + + if options.SkipSearchAttributes { + continue + } + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetIndexedFields(), + ); err != nil { + return err + } + + case []*export.WorkflowExecution: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *export.WorkflowExecution: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHistory(), + ); err != nil { + return err + } + + case *export.WorkflowExecutions: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetItems(), + ); err != nil { + return err + } + + case *failure.ApplicationFailureInfo: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + ); err != nil { + return err + } + + case *failure.CanceledFailureInfo: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + ); err != nil { + return err + } + + case []*failure.Failure: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *failure.Failure: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetApplicationFailureInfo(), + o.GetCanceledFailureInfo(), + o.GetCause(), + o.GetEncodedAttributes(), + o.GetResetWorkflowFailureInfo(), + o.GetTimeoutFailureInfo(), + ); err != nil { + return err + } + + case *failure.ResetWorkflowFailureInfo: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetLastHeartbeatDetails(), + ); err != nil { + return err + } + + case *failure.TimeoutFailureInfo: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetLastHeartbeatDetails(), + ); err != nil { + return err + } + + case *history.ActivityTaskCanceledEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + ); err != nil { + return err + } + + case *history.ActivityTaskCompletedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetResult(), + ); err != nil { + return err + } + + case *history.ActivityTaskFailedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.ActivityTaskScheduledEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + o.GetInput(), + ); err != nil { + return err + } + + case *history.ActivityTaskStartedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetLastFailure(), + ); err != nil { + return err + } + + case *history.ActivityTaskTimedOutEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.ChildWorkflowExecutionCanceledEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + ); err != nil { + return err + } + + case *history.ChildWorkflowExecutionCompletedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetResult(), + ); err != nil { + return err + } + + case *history.ChildWorkflowExecutionFailedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.ChildWorkflowExecutionStartedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + ); err != nil { + return err + } + + case *history.History: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetEvents(), + ); err != nil { + return err + } + + case []*history.HistoryEvent: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *history.HistoryEvent: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetActivityTaskCanceledEventAttributes(), + o.GetActivityTaskCompletedEventAttributes(), + o.GetActivityTaskFailedEventAttributes(), + o.GetActivityTaskScheduledEventAttributes(), + o.GetActivityTaskStartedEventAttributes(), + o.GetActivityTaskTimedOutEventAttributes(), + o.GetChildWorkflowExecutionCanceledEventAttributes(), + o.GetChildWorkflowExecutionCompletedEventAttributes(), + o.GetChildWorkflowExecutionFailedEventAttributes(), + o.GetChildWorkflowExecutionStartedEventAttributes(), + o.GetMarkerRecordedEventAttributes(), + o.GetNexusOperationCanceledEventAttributes(), + o.GetNexusOperationCompletedEventAttributes(), + o.GetNexusOperationFailedEventAttributes(), + o.GetNexusOperationScheduledEventAttributes(), + o.GetNexusOperationTimedOutEventAttributes(), + o.GetSignalExternalWorkflowExecutionInitiatedEventAttributes(), + o.GetStartChildWorkflowExecutionInitiatedEventAttributes(), + o.GetUpsertWorkflowSearchAttributesEventAttributes(), + o.GetUserMetadata(), + o.GetWorkflowExecutionCanceledEventAttributes(), + o.GetWorkflowExecutionCompletedEventAttributes(), + o.GetWorkflowExecutionContinuedAsNewEventAttributes(), + o.GetWorkflowExecutionFailedEventAttributes(), + o.GetWorkflowExecutionSignaledEventAttributes(), + o.GetWorkflowExecutionStartedEventAttributes(), + o.GetWorkflowExecutionTerminatedEventAttributes(), + o.GetWorkflowExecutionUpdateAcceptedEventAttributes(), + o.GetWorkflowExecutionUpdateAdmittedEventAttributes(), + o.GetWorkflowExecutionUpdateCompletedEventAttributes(), + o.GetWorkflowExecutionUpdateRejectedEventAttributes(), + o.GetWorkflowPropertiesModifiedEventAttributes(), + o.GetWorkflowPropertiesModifiedExternallyEventAttributes(), + o.GetWorkflowTaskFailedEventAttributes(), + ); err != nil { + return err + } + + case *history.MarkerRecordedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + o.GetFailure(), + o.GetHeader(), + ); err != nil { + return err + } + + case *history.NexusOperationCanceledEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.NexusOperationCompletedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetResult(), + ); err != nil { + return err + } + + case *history.NexusOperationFailedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.NexusOperationScheduledEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetInput(), + ); err != nil { + return err + } + + case *history.NexusOperationTimedOutEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.SignalExternalWorkflowExecutionInitiatedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + o.GetInput(), + ); err != nil { + return err + } + + case *history.StartChildWorkflowExecutionInitiatedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + o.GetInput(), + o.GetMemo(), + o.GetSearchAttributes(), + ); err != nil { + return err + } + + case *history.UpsertWorkflowSearchAttributesEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetSearchAttributes(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionCanceledEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionCompletedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetResult(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionContinuedAsNewEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + o.GetHeader(), + o.GetInput(), + o.GetLastCompletionResult(), + o.GetMemo(), + o.GetSearchAttributes(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionFailedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionSignaledEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + o.GetInput(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionStartedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetContinuedFailure(), + o.GetHeader(), + o.GetInput(), + o.GetLastCompletionResult(), + o.GetMemo(), + o.GetSearchAttributes(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionTerminatedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionUpdateAcceptedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetAcceptedRequest(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionUpdateAdmittedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetRequest(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionUpdateCompletedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetOutcome(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionUpdateRejectedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + o.GetRejectedRequest(), + ); err != nil { + return err + } + + case *history.WorkflowPropertiesModifiedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetUpsertedMemo(), + ); err != nil { + return err + } + + case *history.WorkflowPropertiesModifiedExternallyEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetUpsertedMemo(), + ); err != nil { + return err + } + + case *history.WorkflowTaskFailedEventAttributes: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *nexus.Request: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetStartOperation(), + ); err != nil { + return err + } + + case *nexus.Response: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetStartOperation(), + ); err != nil { + return err + } + + case *nexus.StartOperationRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetPayload(), + ); err != nil { + return err + } + + case *nexus.StartOperationResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetSyncSuccess(), + ); err != nil { + return err + } + + case *nexus.StartOperationResponse_Sync: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetPayload(), + ); err != nil { + return err + } + + case map[string]*query.WorkflowQuery: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *query.WorkflowQuery: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + o.GetQueryArgs(), + ); err != nil { + return err + } + + case map[string]*query.WorkflowQueryResult: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *query.WorkflowQueryResult: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetAnswer(), + ); err != nil { + return err + } + + case *schedule.Schedule: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetAction(), + ); err != nil { + return err + } + + case *schedule.ScheduleAction: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetStartWorkflow(), + ); err != nil { + return err + } + + case []*schedule.ScheduleListEntry: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *schedule.ScheduleListEntry: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetMemo(), + o.GetSearchAttributes(), + ); err != nil { + return err + } + + case *sdk.UserMetadata: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + o.GetSummary(), + ); err != nil { + return err + } + + case *update.Input: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetArgs(), + o.GetHeader(), + ); err != nil { + return err + } + + case *update.Outcome: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + o.GetSuccess(), + ); err != nil { + return err + } + + case *update.Request: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetInput(), + ); err != nil { + return err + } + + case []*workflow.CallbackInfo: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *workflow.CallbackInfo: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetLastAttemptFailure(), + ); err != nil { + return err + } + + case *workflow.NewWorkflowExecutionInfo: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + o.GetInput(), + o.GetMemo(), + o.GetSearchAttributes(), + o.GetUserMetadata(), + ); err != nil { + return err + } + + case *workflow.NexusOperationCancellationInfo: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetLastAttemptFailure(), + ); err != nil { + return err + } + + case []*workflow.PendingActivityInfo: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *workflow.PendingActivityInfo: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeartbeatDetails(), + o.GetLastFailure(), + ); err != nil { + return err + } + + case []*workflow.PendingNexusOperationInfo: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *workflow.PendingNexusOperationInfo: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetCancellationInfo(), + o.GetLastAttemptFailure(), + ); err != nil { + return err + } + + case *workflow.WorkflowExecutionConfig: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetUserMetadata(), + ); err != nil { + return err + } + + case []*workflow.WorkflowExecutionInfo: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *workflow.WorkflowExecutionInfo: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetMemo(), + o.GetSearchAttributes(), + ); err != nil { + return err + } + + case *workflowservice.CountWorkflowExecutionsResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetGroups(), + ); err != nil { + return err + } + + case []*workflowservice.CountWorkflowExecutionsResponse_AggregationGroup: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *workflowservice.CountWorkflowExecutionsResponse_AggregationGroup: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetGroupValues(), + ); err != nil { + return err + } + + case *workflowservice.CreateScheduleRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetMemo(), + o.GetSchedule(), + o.GetSearchAttributes(), + ); err != nil { + return err + } + + case *workflowservice.DescribeScheduleResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetMemo(), + o.GetSchedule(), + o.GetSearchAttributes(), + ); err != nil { + return err + } + + case *workflowservice.DescribeWorkflowExecutionResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetCallbacks(), + o.GetExecutionConfig(), + o.GetPendingActivities(), + o.GetPendingNexusOperations(), + o.GetWorkflowExecutionInfo(), + ); err != nil { + return err + } + + case *workflowservice.ExecuteMultiOperationRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetOperations(), + ); err != nil { + return err + } + + case []*workflowservice.ExecuteMultiOperationRequest_Operation: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *workflowservice.ExecuteMultiOperationRequest_Operation: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetStartWorkflow(), + o.GetUpdateWorkflow(), + ); err != nil { + return err + } + + case *workflowservice.ExecuteMultiOperationResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetResponses(), + ); err != nil { + return err + } + + case []*workflowservice.ExecuteMultiOperationResponse_Response: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *workflowservice.ExecuteMultiOperationResponse_Response: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetStartWorkflow(), + o.GetUpdateWorkflow(), + ); err != nil { + return err + } + + case *workflowservice.GetWorkflowExecutionHistoryResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHistory(), + ); err != nil { + return err + } + + case *workflowservice.GetWorkflowExecutionHistoryReverseResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHistory(), + ); err != nil { + return err + } + + case *workflowservice.ListArchivedWorkflowExecutionsResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetExecutions(), + ); err != nil { + return err + } + + case *workflowservice.ListClosedWorkflowExecutionsResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetExecutions(), + ); err != nil { + return err + } + + case *workflowservice.ListOpenWorkflowExecutionsResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetExecutions(), + ); err != nil { + return err + } + + case *workflowservice.ListSchedulesResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetSchedules(), + ); err != nil { + return err + } + + case *workflowservice.ListWorkflowExecutionsResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetExecutions(), + ); err != nil { + return err + } + + case []*workflowservice.PollActivityTaskQueueResponse: + for _, x := range o { + if err := visitPayloads(ctx, options, x); err != nil { + return err + } + } + + case *workflowservice.PollActivityTaskQueueResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + o.GetHeartbeatDetails(), + o.GetInput(), + ); err != nil { + return err + } + + case *workflowservice.PollNexusTaskQueueResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetRequest(), + ); err != nil { + return err + } + + case *workflowservice.PollWorkflowExecutionUpdateResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetOutcome(), + ); err != nil { + return err + } + + case *workflowservice.PollWorkflowTaskQueueResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHistory(), + o.GetQueries(), + o.GetQuery(), + ); err != nil { + return err + } + + case *workflowservice.QueryWorkflowRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetQuery(), + ); err != nil { + return err + } + + case *workflowservice.QueryWorkflowResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetQueryResult(), + ); err != nil { + return err + } + + case *workflowservice.RecordActivityTaskHeartbeatByIdRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + ); err != nil { + return err + } + + case *workflowservice.RecordActivityTaskHeartbeatRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + ); err != nil { + return err + } + + case *workflowservice.RespondActivityTaskCanceledByIdRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + ); err != nil { + return err + } + + case *workflowservice.RespondActivityTaskCanceledRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + ); err != nil { + return err + } + + case *workflowservice.RespondActivityTaskCompletedByIdRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetResult(), + ); err != nil { + return err + } + + case *workflowservice.RespondActivityTaskCompletedRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetResult(), + ); err != nil { + return err + } + + case *workflowservice.RespondActivityTaskFailedByIdRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + o.GetLastHeartbeatDetails(), + ); err != nil { + return err + } + + case *workflowservice.RespondActivityTaskFailedByIdResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailures(), + ); err != nil { + return err + } + + case *workflowservice.RespondActivityTaskFailedRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + o.GetLastHeartbeatDetails(), + ); err != nil { + return err + } + + case *workflowservice.RespondActivityTaskFailedResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailures(), + ); err != nil { + return err + } + + case *workflowservice.RespondNexusTaskCompletedRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetResponse(), + ); err != nil { + return err + } + + case *workflowservice.RespondQueryTaskCompletedRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetQueryResult(), + ); err != nil { + return err + } + + case *workflowservice.RespondWorkflowTaskCompletedRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetCommands(), + o.GetQueryResults(), + ); err != nil { + return err + } + + case *workflowservice.RespondWorkflowTaskCompletedResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetActivityTasks(), + o.GetWorkflowTask(), + ); err != nil { + return err + } + + case *workflowservice.RespondWorkflowTaskFailedRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *workflowservice.ScanWorkflowExecutionsResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetExecutions(), + ); err != nil { + return err + } + + case *workflowservice.SignalWithStartWorkflowExecutionRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + o.GetInput(), + o.GetMemo(), + o.GetSearchAttributes(), + o.GetSignalInput(), + o.GetUserMetadata(), + ); err != nil { + return err + } + + case *workflowservice.SignalWorkflowExecutionRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetHeader(), + o.GetInput(), + ); err != nil { + return err + } + + case *workflowservice.StartBatchOperationRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetSignalOperation(), + o.GetTerminationOperation(), + ); err != nil { + return err + } + + case *workflowservice.StartWorkflowExecutionRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetContinuedFailure(), + o.GetHeader(), + o.GetInput(), + o.GetLastCompletionResult(), + o.GetMemo(), + o.GetSearchAttributes(), + o.GetUserMetadata(), + ); err != nil { + return err + } + + case *workflowservice.StartWorkflowExecutionResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetEagerWorkflowTask(), + ); err != nil { + return err + } + + case *workflowservice.TerminateWorkflowExecutionRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetDetails(), + ); err != nil { + return err + } + + case *workflowservice.UpdateScheduleRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetSchedule(), + o.GetSearchAttributes(), + ); err != nil { + return err + } + + case *workflowservice.UpdateWorkflowExecutionRequest: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetRequest(), + ); err != nil { + return err + } + + case *workflowservice.UpdateWorkflowExecutionResponse: + + if o == nil { + continue + } + ctx.Parent = o + if err := visitPayloads( + ctx, + options, + o.GetOutcome(), + ); err != nil { + return err + } + + } + } + + return nil +} + +func visitFailures(ctx *VisitFailuresContext, options *VisitFailuresOptions, objs ...interface{}) error { + for _, obj := range objs { + switch o := obj.(type) { + case *failure.Failure: + if o == nil { + continue + } + if err := options.Visitor(ctx, o); err != nil { + return err + } + if err := visitFailures(ctx, options, o.GetCause()); err != nil { + return err + } + + case []*command.Command: + for _, x := range o { + if err := visitFailures(ctx, options, x); err != nil { + return err + } + } + + case *command.Command: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetContinueAsNewWorkflowExecutionCommandAttributes(), + o.GetFailWorkflowExecutionCommandAttributes(), + o.GetRecordMarkerCommandAttributes(), + ); err != nil { + return err + } + + case *command.ContinueAsNewWorkflowExecutionCommandAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *command.FailWorkflowExecutionCommandAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *command.RecordMarkerCommandAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case []*export.WorkflowExecution: + for _, x := range o { + if err := visitFailures(ctx, options, x); err != nil { + return err + } + } + + case *export.WorkflowExecution: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetHistory(), + ); err != nil { + return err + } + + case *export.WorkflowExecutions: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetItems(), + ); err != nil { + return err + } + + case *history.ActivityTaskFailedEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.ActivityTaskStartedEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetLastFailure(), + ); err != nil { + return err + } + + case *history.ActivityTaskTimedOutEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.ChildWorkflowExecutionFailedEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.History: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetEvents(), + ); err != nil { + return err + } + + case []*history.HistoryEvent: + for _, x := range o { + if err := visitFailures(ctx, options, x); err != nil { + return err + } + } + + case *history.HistoryEvent: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetActivityTaskFailedEventAttributes(), + o.GetActivityTaskStartedEventAttributes(), + o.GetActivityTaskTimedOutEventAttributes(), + o.GetChildWorkflowExecutionFailedEventAttributes(), + o.GetMarkerRecordedEventAttributes(), + o.GetNexusOperationCanceledEventAttributes(), + o.GetNexusOperationFailedEventAttributes(), + o.GetNexusOperationTimedOutEventAttributes(), + o.GetWorkflowExecutionContinuedAsNewEventAttributes(), + o.GetWorkflowExecutionFailedEventAttributes(), + o.GetWorkflowExecutionStartedEventAttributes(), + o.GetWorkflowExecutionUpdateCompletedEventAttributes(), + o.GetWorkflowExecutionUpdateRejectedEventAttributes(), + o.GetWorkflowTaskFailedEventAttributes(), + ); err != nil { + return err + } + + case *history.MarkerRecordedEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.NexusOperationCanceledEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.NexusOperationFailedEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.NexusOperationTimedOutEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionContinuedAsNewEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionFailedEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionStartedEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetContinuedFailure(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionUpdateCompletedEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetOutcome(), + ); err != nil { + return err + } + + case *history.WorkflowExecutionUpdateRejectedEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *history.WorkflowTaskFailedEventAttributes: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *update.Outcome: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case []*workflow.CallbackInfo: + for _, x := range o { + if err := visitFailures(ctx, options, x); err != nil { + return err + } + } + + case *workflow.CallbackInfo: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetLastAttemptFailure(), + ); err != nil { + return err + } + + case *workflow.NexusOperationCancellationInfo: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetLastAttemptFailure(), + ); err != nil { + return err + } + + case []*workflow.PendingActivityInfo: + for _, x := range o { + if err := visitFailures(ctx, options, x); err != nil { + return err + } + } + + case *workflow.PendingActivityInfo: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetLastFailure(), + ); err != nil { + return err + } + + case []*workflow.PendingNexusOperationInfo: + for _, x := range o { + if err := visitFailures(ctx, options, x); err != nil { + return err + } + } + + case *workflow.PendingNexusOperationInfo: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetCancellationInfo(), + o.GetLastAttemptFailure(), + ); err != nil { + return err + } + + case *workflowservice.DescribeWorkflowExecutionResponse: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetCallbacks(), + o.GetPendingActivities(), + o.GetPendingNexusOperations(), + ); err != nil { + return err + } + + case *workflowservice.ExecuteMultiOperationRequest: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetOperations(), + ); err != nil { + return err + } + + case []*workflowservice.ExecuteMultiOperationRequest_Operation: + for _, x := range o { + if err := visitFailures(ctx, options, x); err != nil { + return err + } + } + + case *workflowservice.ExecuteMultiOperationRequest_Operation: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetStartWorkflow(), + ); err != nil { + return err + } + + case *workflowservice.ExecuteMultiOperationResponse: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetResponses(), + ); err != nil { + return err + } + + case []*workflowservice.ExecuteMultiOperationResponse_Response: + for _, x := range o { + if err := visitFailures(ctx, options, x); err != nil { + return err + } + } + + case *workflowservice.ExecuteMultiOperationResponse_Response: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetStartWorkflow(), + o.GetUpdateWorkflow(), + ); err != nil { + return err + } + + case *workflowservice.GetWorkflowExecutionHistoryResponse: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetHistory(), + ); err != nil { + return err + } + + case *workflowservice.GetWorkflowExecutionHistoryReverseResponse: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetHistory(), + ); err != nil { + return err + } + + case *workflowservice.PollWorkflowExecutionUpdateResponse: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetOutcome(), + ); err != nil { + return err + } + + case *workflowservice.PollWorkflowTaskQueueResponse: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetHistory(), + ); err != nil { + return err + } + + case *workflowservice.RespondActivityTaskFailedByIdRequest: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *workflowservice.RespondActivityTaskFailedByIdResponse: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailures(), + ); err != nil { + return err + } + + case *workflowservice.RespondActivityTaskFailedRequest: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *workflowservice.RespondActivityTaskFailedResponse: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailures(), + ); err != nil { + return err + } + + case *workflowservice.RespondWorkflowTaskCompletedRequest: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetCommands(), + ); err != nil { + return err + } + + case *workflowservice.RespondWorkflowTaskCompletedResponse: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetWorkflowTask(), + ); err != nil { + return err + } + + case *workflowservice.RespondWorkflowTaskFailedRequest: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetFailure(), + ); err != nil { + return err + } + + case *workflowservice.StartWorkflowExecutionRequest: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetContinuedFailure(), + ); err != nil { + return err + } + + case *workflowservice.StartWorkflowExecutionResponse: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetEagerWorkflowTask(), + ); err != nil { + return err + } + + case *workflowservice.UpdateWorkflowExecutionResponse: + if o == nil { + continue + } + ctx.Parent = o + if err := visitFailures( + ctx, + options, + o.GetOutcome(), + ); err != nil { + return err + } + + } + } + + return nil +} diff --git a/vendor/go.temporal.io/api/proxy/service.go b/vendor/go.temporal.io/api/proxy/service.go new file mode 100644 index 00000000000..0179dc6f06f --- /dev/null +++ b/vendor/go.temporal.io/api/proxy/service.go @@ -0,0 +1,307 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by proxygenerator; DO NOT EDIT. + +package proxy + +import ( + "context" + + "go.temporal.io/api/workflowservice/v1" +) + +// WorkflowServiceProxyOptions provides options for configuring a WorkflowServiceProxyServer. +// Client is a WorkflowServiceClient used to forward requests received by the server to the +// Temporal Frontend. +type WorkflowServiceProxyOptions struct { + Client workflowservice.WorkflowServiceClient + DisableHeaderForwarding bool +} + +type workflowServiceProxyServer struct { + workflowservice.UnimplementedWorkflowServiceServer + client workflowservice.WorkflowServiceClient + disableHeaderForwarding bool +} + +// NewWorkflowServiceProxyServer creates a WorkflowServiceServer suitable for registering with a gRPC Server. Requests will +// be forwarded to the passed in WorkflowService Client. gRPC interceptors can be added on the Server or Client to adjust +// requests and responses. +func NewWorkflowServiceProxyServer(options WorkflowServiceProxyOptions) (workflowservice.WorkflowServiceServer, error) { + return &workflowServiceProxyServer{ + client: options.Client, + disableHeaderForwarding: options.DisableHeaderForwarding, + }, nil +} + +func (s *workflowServiceProxyServer) CountWorkflowExecutions(ctx context.Context, in0 *workflowservice.CountWorkflowExecutionsRequest) (*workflowservice.CountWorkflowExecutionsResponse, error) { + return s.client.CountWorkflowExecutions(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) CreateSchedule(ctx context.Context, in0 *workflowservice.CreateScheduleRequest) (*workflowservice.CreateScheduleResponse, error) { + return s.client.CreateSchedule(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) DeleteSchedule(ctx context.Context, in0 *workflowservice.DeleteScheduleRequest) (*workflowservice.DeleteScheduleResponse, error) { + return s.client.DeleteSchedule(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) DeleteWorkflowExecution(ctx context.Context, in0 *workflowservice.DeleteWorkflowExecutionRequest) (*workflowservice.DeleteWorkflowExecutionResponse, error) { + return s.client.DeleteWorkflowExecution(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) DeprecateNamespace(ctx context.Context, in0 *workflowservice.DeprecateNamespaceRequest) (*workflowservice.DeprecateNamespaceResponse, error) { + return s.client.DeprecateNamespace(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) DescribeBatchOperation(ctx context.Context, in0 *workflowservice.DescribeBatchOperationRequest) (*workflowservice.DescribeBatchOperationResponse, error) { + return s.client.DescribeBatchOperation(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) DescribeNamespace(ctx context.Context, in0 *workflowservice.DescribeNamespaceRequest) (*workflowservice.DescribeNamespaceResponse, error) { + return s.client.DescribeNamespace(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) DescribeSchedule(ctx context.Context, in0 *workflowservice.DescribeScheduleRequest) (*workflowservice.DescribeScheduleResponse, error) { + return s.client.DescribeSchedule(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) DescribeTaskQueue(ctx context.Context, in0 *workflowservice.DescribeTaskQueueRequest) (*workflowservice.DescribeTaskQueueResponse, error) { + return s.client.DescribeTaskQueue(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) DescribeWorkflowExecution(ctx context.Context, in0 *workflowservice.DescribeWorkflowExecutionRequest) (*workflowservice.DescribeWorkflowExecutionResponse, error) { + return s.client.DescribeWorkflowExecution(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) ExecuteMultiOperation(ctx context.Context, in0 *workflowservice.ExecuteMultiOperationRequest) (*workflowservice.ExecuteMultiOperationResponse, error) { + return s.client.ExecuteMultiOperation(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) GetClusterInfo(ctx context.Context, in0 *workflowservice.GetClusterInfoRequest) (*workflowservice.GetClusterInfoResponse, error) { + return s.client.GetClusterInfo(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) GetSearchAttributes(ctx context.Context, in0 *workflowservice.GetSearchAttributesRequest) (*workflowservice.GetSearchAttributesResponse, error) { + return s.client.GetSearchAttributes(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) GetSystemInfo(ctx context.Context, in0 *workflowservice.GetSystemInfoRequest) (*workflowservice.GetSystemInfoResponse, error) { + return s.client.GetSystemInfo(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) GetWorkerBuildIdCompatibility(ctx context.Context, in0 *workflowservice.GetWorkerBuildIdCompatibilityRequest) (*workflowservice.GetWorkerBuildIdCompatibilityResponse, error) { + return s.client.GetWorkerBuildIdCompatibility(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) GetWorkerTaskReachability(ctx context.Context, in0 *workflowservice.GetWorkerTaskReachabilityRequest) (*workflowservice.GetWorkerTaskReachabilityResponse, error) { + return s.client.GetWorkerTaskReachability(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) GetWorkerVersioningRules(ctx context.Context, in0 *workflowservice.GetWorkerVersioningRulesRequest) (*workflowservice.GetWorkerVersioningRulesResponse, error) { + return s.client.GetWorkerVersioningRules(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) GetWorkflowExecutionHistory(ctx context.Context, in0 *workflowservice.GetWorkflowExecutionHistoryRequest) (*workflowservice.GetWorkflowExecutionHistoryResponse, error) { + return s.client.GetWorkflowExecutionHistory(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) GetWorkflowExecutionHistoryReverse(ctx context.Context, in0 *workflowservice.GetWorkflowExecutionHistoryReverseRequest) (*workflowservice.GetWorkflowExecutionHistoryReverseResponse, error) { + return s.client.GetWorkflowExecutionHistoryReverse(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) ListArchivedWorkflowExecutions(ctx context.Context, in0 *workflowservice.ListArchivedWorkflowExecutionsRequest) (*workflowservice.ListArchivedWorkflowExecutionsResponse, error) { + return s.client.ListArchivedWorkflowExecutions(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) ListBatchOperations(ctx context.Context, in0 *workflowservice.ListBatchOperationsRequest) (*workflowservice.ListBatchOperationsResponse, error) { + return s.client.ListBatchOperations(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) ListClosedWorkflowExecutions(ctx context.Context, in0 *workflowservice.ListClosedWorkflowExecutionsRequest) (*workflowservice.ListClosedWorkflowExecutionsResponse, error) { + return s.client.ListClosedWorkflowExecutions(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) ListNamespaces(ctx context.Context, in0 *workflowservice.ListNamespacesRequest) (*workflowservice.ListNamespacesResponse, error) { + return s.client.ListNamespaces(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) ListOpenWorkflowExecutions(ctx context.Context, in0 *workflowservice.ListOpenWorkflowExecutionsRequest) (*workflowservice.ListOpenWorkflowExecutionsResponse, error) { + return s.client.ListOpenWorkflowExecutions(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) ListScheduleMatchingTimes(ctx context.Context, in0 *workflowservice.ListScheduleMatchingTimesRequest) (*workflowservice.ListScheduleMatchingTimesResponse, error) { + return s.client.ListScheduleMatchingTimes(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) ListSchedules(ctx context.Context, in0 *workflowservice.ListSchedulesRequest) (*workflowservice.ListSchedulesResponse, error) { + return s.client.ListSchedules(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) ListTaskQueuePartitions(ctx context.Context, in0 *workflowservice.ListTaskQueuePartitionsRequest) (*workflowservice.ListTaskQueuePartitionsResponse, error) { + return s.client.ListTaskQueuePartitions(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) ListWorkflowExecutions(ctx context.Context, in0 *workflowservice.ListWorkflowExecutionsRequest) (*workflowservice.ListWorkflowExecutionsResponse, error) { + return s.client.ListWorkflowExecutions(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) PatchSchedule(ctx context.Context, in0 *workflowservice.PatchScheduleRequest) (*workflowservice.PatchScheduleResponse, error) { + return s.client.PatchSchedule(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) PollActivityTaskQueue(ctx context.Context, in0 *workflowservice.PollActivityTaskQueueRequest) (*workflowservice.PollActivityTaskQueueResponse, error) { + return s.client.PollActivityTaskQueue(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) PollNexusTaskQueue(ctx context.Context, in0 *workflowservice.PollNexusTaskQueueRequest) (*workflowservice.PollNexusTaskQueueResponse, error) { + return s.client.PollNexusTaskQueue(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) PollWorkflowExecutionUpdate(ctx context.Context, in0 *workflowservice.PollWorkflowExecutionUpdateRequest) (*workflowservice.PollWorkflowExecutionUpdateResponse, error) { + return s.client.PollWorkflowExecutionUpdate(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) PollWorkflowTaskQueue(ctx context.Context, in0 *workflowservice.PollWorkflowTaskQueueRequest) (*workflowservice.PollWorkflowTaskQueueResponse, error) { + return s.client.PollWorkflowTaskQueue(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) QueryWorkflow(ctx context.Context, in0 *workflowservice.QueryWorkflowRequest) (*workflowservice.QueryWorkflowResponse, error) { + return s.client.QueryWorkflow(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RecordActivityTaskHeartbeat(ctx context.Context, in0 *workflowservice.RecordActivityTaskHeartbeatRequest) (*workflowservice.RecordActivityTaskHeartbeatResponse, error) { + return s.client.RecordActivityTaskHeartbeat(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RecordActivityTaskHeartbeatById(ctx context.Context, in0 *workflowservice.RecordActivityTaskHeartbeatByIdRequest) (*workflowservice.RecordActivityTaskHeartbeatByIdResponse, error) { + return s.client.RecordActivityTaskHeartbeatById(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RegisterNamespace(ctx context.Context, in0 *workflowservice.RegisterNamespaceRequest) (*workflowservice.RegisterNamespaceResponse, error) { + return s.client.RegisterNamespace(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RequestCancelWorkflowExecution(ctx context.Context, in0 *workflowservice.RequestCancelWorkflowExecutionRequest) (*workflowservice.RequestCancelWorkflowExecutionResponse, error) { + return s.client.RequestCancelWorkflowExecution(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) ResetStickyTaskQueue(ctx context.Context, in0 *workflowservice.ResetStickyTaskQueueRequest) (*workflowservice.ResetStickyTaskQueueResponse, error) { + return s.client.ResetStickyTaskQueue(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) ResetWorkflowExecution(ctx context.Context, in0 *workflowservice.ResetWorkflowExecutionRequest) (*workflowservice.ResetWorkflowExecutionResponse, error) { + return s.client.ResetWorkflowExecution(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RespondActivityTaskCanceled(ctx context.Context, in0 *workflowservice.RespondActivityTaskCanceledRequest) (*workflowservice.RespondActivityTaskCanceledResponse, error) { + return s.client.RespondActivityTaskCanceled(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RespondActivityTaskCanceledById(ctx context.Context, in0 *workflowservice.RespondActivityTaskCanceledByIdRequest) (*workflowservice.RespondActivityTaskCanceledByIdResponse, error) { + return s.client.RespondActivityTaskCanceledById(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RespondActivityTaskCompleted(ctx context.Context, in0 *workflowservice.RespondActivityTaskCompletedRequest) (*workflowservice.RespondActivityTaskCompletedResponse, error) { + return s.client.RespondActivityTaskCompleted(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RespondActivityTaskCompletedById(ctx context.Context, in0 *workflowservice.RespondActivityTaskCompletedByIdRequest) (*workflowservice.RespondActivityTaskCompletedByIdResponse, error) { + return s.client.RespondActivityTaskCompletedById(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RespondActivityTaskFailed(ctx context.Context, in0 *workflowservice.RespondActivityTaskFailedRequest) (*workflowservice.RespondActivityTaskFailedResponse, error) { + return s.client.RespondActivityTaskFailed(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RespondActivityTaskFailedById(ctx context.Context, in0 *workflowservice.RespondActivityTaskFailedByIdRequest) (*workflowservice.RespondActivityTaskFailedByIdResponse, error) { + return s.client.RespondActivityTaskFailedById(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RespondNexusTaskCompleted(ctx context.Context, in0 *workflowservice.RespondNexusTaskCompletedRequest) (*workflowservice.RespondNexusTaskCompletedResponse, error) { + return s.client.RespondNexusTaskCompleted(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RespondNexusTaskFailed(ctx context.Context, in0 *workflowservice.RespondNexusTaskFailedRequest) (*workflowservice.RespondNexusTaskFailedResponse, error) { + return s.client.RespondNexusTaskFailed(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RespondQueryTaskCompleted(ctx context.Context, in0 *workflowservice.RespondQueryTaskCompletedRequest) (*workflowservice.RespondQueryTaskCompletedResponse, error) { + return s.client.RespondQueryTaskCompleted(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RespondWorkflowTaskCompleted(ctx context.Context, in0 *workflowservice.RespondWorkflowTaskCompletedRequest) (*workflowservice.RespondWorkflowTaskCompletedResponse, error) { + return s.client.RespondWorkflowTaskCompleted(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) RespondWorkflowTaskFailed(ctx context.Context, in0 *workflowservice.RespondWorkflowTaskFailedRequest) (*workflowservice.RespondWorkflowTaskFailedResponse, error) { + return s.client.RespondWorkflowTaskFailed(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) ScanWorkflowExecutions(ctx context.Context, in0 *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error) { + return s.client.ScanWorkflowExecutions(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) SignalWithStartWorkflowExecution(ctx context.Context, in0 *workflowservice.SignalWithStartWorkflowExecutionRequest) (*workflowservice.SignalWithStartWorkflowExecutionResponse, error) { + return s.client.SignalWithStartWorkflowExecution(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) SignalWorkflowExecution(ctx context.Context, in0 *workflowservice.SignalWorkflowExecutionRequest) (*workflowservice.SignalWorkflowExecutionResponse, error) { + return s.client.SignalWorkflowExecution(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) StartBatchOperation(ctx context.Context, in0 *workflowservice.StartBatchOperationRequest) (*workflowservice.StartBatchOperationResponse, error) { + return s.client.StartBatchOperation(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) StartWorkflowExecution(ctx context.Context, in0 *workflowservice.StartWorkflowExecutionRequest) (*workflowservice.StartWorkflowExecutionResponse, error) { + return s.client.StartWorkflowExecution(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) StopBatchOperation(ctx context.Context, in0 *workflowservice.StopBatchOperationRequest) (*workflowservice.StopBatchOperationResponse, error) { + return s.client.StopBatchOperation(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) TerminateWorkflowExecution(ctx context.Context, in0 *workflowservice.TerminateWorkflowExecutionRequest) (*workflowservice.TerminateWorkflowExecutionResponse, error) { + return s.client.TerminateWorkflowExecution(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) UpdateNamespace(ctx context.Context, in0 *workflowservice.UpdateNamespaceRequest) (*workflowservice.UpdateNamespaceResponse, error) { + return s.client.UpdateNamespace(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) UpdateSchedule(ctx context.Context, in0 *workflowservice.UpdateScheduleRequest) (*workflowservice.UpdateScheduleResponse, error) { + return s.client.UpdateSchedule(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) UpdateWorkerBuildIdCompatibility(ctx context.Context, in0 *workflowservice.UpdateWorkerBuildIdCompatibilityRequest) (*workflowservice.UpdateWorkerBuildIdCompatibilityResponse, error) { + return s.client.UpdateWorkerBuildIdCompatibility(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) UpdateWorkerVersioningRules(ctx context.Context, in0 *workflowservice.UpdateWorkerVersioningRulesRequest) (*workflowservice.UpdateWorkerVersioningRulesResponse, error) { + return s.client.UpdateWorkerVersioningRules(s.reqCtx(ctx), in0) +} + +func (s *workflowServiceProxyServer) UpdateWorkflowExecution(ctx context.Context, in0 *workflowservice.UpdateWorkflowExecutionRequest) (*workflowservice.UpdateWorkflowExecutionResponse, error) { + return s.client.UpdateWorkflowExecution(s.reqCtx(ctx), in0) +} diff --git a/vendor/go.temporal.io/api/proxy/service_util.go b/vendor/go.temporal.io/api/proxy/service_util.go new file mode 100644 index 00000000000..9c213ada1d9 --- /dev/null +++ b/vendor/go.temporal.io/api/proxy/service_util.go @@ -0,0 +1,60 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package proxy + +import ( + "context" + + "google.golang.org/grpc/metadata" +) + +func (s *workflowServiceProxyServer) reqCtx(ctx context.Context) context.Context { + if s.disableHeaderForwarding { + return ctx + } + + // Copy incoming header to outgoing if not already present in outgoing. We + // have confirmed in gRPC that incoming is a copy so we can mutate it. + incoming, _ := metadata.FromIncomingContext(ctx) + + // Remove common headers and if there's nothing left, return early + incoming.Delete("user-agent") + incoming.Delete(":authority") + incoming.Delete("content-type") + if len(incoming) == 0 { + return ctx + } + + // Put all incoming on outgoing if they are not already there. We have + // confirmed in gRPC that outgoing is a copy so we can mutate it. + outgoing, _ := metadata.FromOutgoingContext(ctx) + if outgoing == nil { + outgoing = metadata.MD{} + } + for k, v := range incoming { + if len(outgoing.Get(k)) == 0 { + outgoing.Set(k, v...) + } + } + return metadata.NewOutgoingContext(ctx, outgoing) +} diff --git a/vendor/go.temporal.io/api/query/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/query/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..af1a984c69c --- /dev/null +++ b/vendor/go.temporal.io/api/query/v1/message.go-helpers.pb.go @@ -0,0 +1,139 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package query + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type WorkflowQuery to the protobuf v3 wire format +func (val *WorkflowQuery) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowQuery from the protobuf v3 wire format +func (val *WorkflowQuery) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowQuery) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowQuery values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowQuery) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowQuery + switch t := that.(type) { + case *WorkflowQuery: + that1 = t + case WorkflowQuery: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowQueryResult to the protobuf v3 wire format +func (val *WorkflowQueryResult) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowQueryResult from the protobuf v3 wire format +func (val *WorkflowQueryResult) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowQueryResult) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowQueryResult values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowQueryResult) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowQueryResult + switch t := that.(type) { + case *WorkflowQueryResult: + that1 = t + case WorkflowQueryResult: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type QueryRejected to the protobuf v3 wire format +func (val *QueryRejected) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type QueryRejected from the protobuf v3 wire format +func (val *QueryRejected) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *QueryRejected) Size() int { + return proto.Size(val) +} + +// Equal returns whether two QueryRejected values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *QueryRejected) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *QueryRejected + switch t := that.(type) { + case *QueryRejected: + that1 = t + case QueryRejected: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/query/v1/message.pb.go b/vendor/go.temporal.io/api/query/v1/message.pb.go new file mode 100644 index 00000000000..79e36ac7de1 --- /dev/null +++ b/vendor/go.temporal.io/api/query/v1/message.pb.go @@ -0,0 +1,379 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/query/v1/message.proto + +package query + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/common/v1" + v11 "go.temporal.io/api/enums/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// See https://docs.temporal.io/docs/concepts/queries/ +type WorkflowQuery struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The workflow-author-defined identifier of the query. Typically a function name. + QueryType string `protobuf:"bytes,1,opt,name=query_type,json=queryType,proto3" json:"query_type,omitempty"` + // Serialized arguments that will be provided to the query handler. + QueryArgs *v1.Payloads `protobuf:"bytes,2,opt,name=query_args,json=queryArgs,proto3" json:"query_args,omitempty"` + // Headers that were passed by the caller of the query and copied by temporal + // server into the workflow task. + Header *v1.Header `protobuf:"bytes,3,opt,name=header,proto3" json:"header,omitempty"` +} + +func (x *WorkflowQuery) Reset() { + *x = WorkflowQuery{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_query_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowQuery) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowQuery) ProtoMessage() {} + +func (x *WorkflowQuery) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_query_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowQuery.ProtoReflect.Descriptor instead. +func (*WorkflowQuery) Descriptor() ([]byte, []int) { + return file_temporal_api_query_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *WorkflowQuery) GetQueryType() string { + if x != nil { + return x.QueryType + } + return "" +} + +func (x *WorkflowQuery) GetQueryArgs() *v1.Payloads { + if x != nil { + return x.QueryArgs + } + return nil +} + +func (x *WorkflowQuery) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +// Answer to a `WorkflowQuery` +type WorkflowQueryResult struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Did the query succeed or fail? + ResultType v11.QueryResultType `protobuf:"varint,1,opt,name=result_type,json=resultType,proto3,enum=temporal.api.enums.v1.QueryResultType" json:"result_type,omitempty"` + // Set when the query succeeds with the results + Answer *v1.Payloads `protobuf:"bytes,2,opt,name=answer,proto3" json:"answer,omitempty"` + // Mutually exclusive with `answer`. Set when the query fails. + ErrorMessage string `protobuf:"bytes,3,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"` +} + +func (x *WorkflowQueryResult) Reset() { + *x = WorkflowQueryResult{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_query_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowQueryResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowQueryResult) ProtoMessage() {} + +func (x *WorkflowQueryResult) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_query_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowQueryResult.ProtoReflect.Descriptor instead. +func (*WorkflowQueryResult) Descriptor() ([]byte, []int) { + return file_temporal_api_query_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *WorkflowQueryResult) GetResultType() v11.QueryResultType { + if x != nil { + return x.ResultType + } + return v11.QueryResultType(0) +} + +func (x *WorkflowQueryResult) GetAnswer() *v1.Payloads { + if x != nil { + return x.Answer + } + return nil +} + +func (x *WorkflowQueryResult) GetErrorMessage() string { + if x != nil { + return x.ErrorMessage + } + return "" +} + +type QueryRejected struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status v11.WorkflowExecutionStatus `protobuf:"varint,1,opt,name=status,proto3,enum=temporal.api.enums.v1.WorkflowExecutionStatus" json:"status,omitempty"` +} + +func (x *QueryRejected) Reset() { + *x = QueryRejected{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_query_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryRejected) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryRejected) ProtoMessage() {} + +func (x *QueryRejected) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_query_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryRejected.ProtoReflect.Descriptor instead. +func (*QueryRejected) Descriptor() ([]byte, []int) { + return file_temporal_api_query_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *QueryRejected) GetStatus() v11.WorkflowExecutionStatus { + if x != nil { + return x.Status + } + return v11.WorkflowExecutionStatus(0) +} + +var File_temporal_api_query_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_query_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x23, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x1a, 0x21, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, + 0x2f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, + 0x31, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x1a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xb3, 0x01, 0x0a, 0x0d, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x21, 0x0a, 0x0a, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x43, 0x0a, 0x0a, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x5f, 0x61, 0x72, 0x67, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x09, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x41, 0x72, 0x67, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x68, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x42, 0x02, 0x68, 0x00, 0x22, 0xc9, 0x01, 0x0a, 0x13, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x4b, 0x0a, 0x0b, 0x72, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x26, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x54, 0x79, + 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3c, 0x0a, 0x06, 0x61, 0x6e, 0x73, 0x77, 0x65, 0x72, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x06, 0x61, 0x6e, 0x73, 0x77, 0x65, 0x72, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x27, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x5b, 0x0a, 0x0d, 0x51, 0x75, 0x65, + 0x72, 0x79, 0x52, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x12, 0x4a, 0x0a, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x42, 0x84, 0x01, 0x0a, 0x18, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x42, 0x0c, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x21, + 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2f, 0x76, 0x31, 0x3b, 0x71, 0x75, 0x65, 0x72, 0x79, + 0xaa, 0x02, 0x17, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, + 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x1a, 0x54, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x51, 0x75, 0x65, 0x72, + 0x79, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_query_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_query_v1_message_proto_rawDescData = file_temporal_api_query_v1_message_proto_rawDesc +) + +func file_temporal_api_query_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_query_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_query_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_query_v1_message_proto_rawDescData) + }) + return file_temporal_api_query_v1_message_proto_rawDescData +} + +var file_temporal_api_query_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 3) +var file_temporal_api_query_v1_message_proto_goTypes = []any{ + (*WorkflowQuery)(nil), // 0: temporal.api.query.v1.WorkflowQuery + (*WorkflowQueryResult)(nil), // 1: temporal.api.query.v1.WorkflowQueryResult + (*QueryRejected)(nil), // 2: temporal.api.query.v1.QueryRejected + (*v1.Payloads)(nil), // 3: temporal.api.common.v1.Payloads + (*v1.Header)(nil), // 4: temporal.api.common.v1.Header + (v11.QueryResultType)(0), // 5: temporal.api.enums.v1.QueryResultType + (v11.WorkflowExecutionStatus)(0), // 6: temporal.api.enums.v1.WorkflowExecutionStatus +} +var file_temporal_api_query_v1_message_proto_depIdxs = []int32{ + 3, // 0: temporal.api.query.v1.WorkflowQuery.query_args:type_name -> temporal.api.common.v1.Payloads + 4, // 1: temporal.api.query.v1.WorkflowQuery.header:type_name -> temporal.api.common.v1.Header + 5, // 2: temporal.api.query.v1.WorkflowQueryResult.result_type:type_name -> temporal.api.enums.v1.QueryResultType + 3, // 3: temporal.api.query.v1.WorkflowQueryResult.answer:type_name -> temporal.api.common.v1.Payloads + 6, // 4: temporal.api.query.v1.QueryRejected.status:type_name -> temporal.api.enums.v1.WorkflowExecutionStatus + 5, // [5:5] is the sub-list for method output_type + 5, // [5:5] is the sub-list for method input_type + 5, // [5:5] is the sub-list for extension type_name + 5, // [5:5] is the sub-list for extension extendee + 0, // [0:5] is the sub-list for field type_name +} + +func init() { file_temporal_api_query_v1_message_proto_init() } +func file_temporal_api_query_v1_message_proto_init() { + if File_temporal_api_query_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_query_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowQuery); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_query_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowQueryResult); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_query_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*QueryRejected); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_query_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 3, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_query_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_query_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_query_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_query_v1_message_proto = out.File + file_temporal_api_query_v1_message_proto_rawDesc = nil + file_temporal_api_query_v1_message_proto_goTypes = nil + file_temporal_api_query_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/replication/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/replication/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..2452ca45b4c --- /dev/null +++ b/vendor/go.temporal.io/api/replication/v1/message.go-helpers.pb.go @@ -0,0 +1,139 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package replication + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type ClusterReplicationConfig to the protobuf v3 wire format +func (val *ClusterReplicationConfig) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ClusterReplicationConfig from the protobuf v3 wire format +func (val *ClusterReplicationConfig) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ClusterReplicationConfig) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ClusterReplicationConfig values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ClusterReplicationConfig) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ClusterReplicationConfig + switch t := that.(type) { + case *ClusterReplicationConfig: + that1 = t + case ClusterReplicationConfig: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NamespaceReplicationConfig to the protobuf v3 wire format +func (val *NamespaceReplicationConfig) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NamespaceReplicationConfig from the protobuf v3 wire format +func (val *NamespaceReplicationConfig) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NamespaceReplicationConfig) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NamespaceReplicationConfig values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NamespaceReplicationConfig) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NamespaceReplicationConfig + switch t := that.(type) { + case *NamespaceReplicationConfig: + that1 = t + case NamespaceReplicationConfig: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type FailoverStatus to the protobuf v3 wire format +func (val *FailoverStatus) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type FailoverStatus from the protobuf v3 wire format +func (val *FailoverStatus) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *FailoverStatus) Size() int { + return proto.Size(val) +} + +// Equal returns whether two FailoverStatus values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *FailoverStatus) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *FailoverStatus + switch t := that.(type) { + case *FailoverStatus: + that1 = t + case FailoverStatus: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/replication/v1/message.pb.go b/vendor/go.temporal.io/api/replication/v1/message.pb.go new file mode 100644 index 00000000000..1300bfed87b --- /dev/null +++ b/vendor/go.temporal.io/api/replication/v1/message.pb.go @@ -0,0 +1,358 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/replication/v1/message.proto + +package replication + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/enums/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type ClusterReplicationConfig struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ClusterName string `protobuf:"bytes,1,opt,name=cluster_name,json=clusterName,proto3" json:"cluster_name,omitempty"` +} + +func (x *ClusterReplicationConfig) Reset() { + *x = ClusterReplicationConfig{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_replication_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ClusterReplicationConfig) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ClusterReplicationConfig) ProtoMessage() {} + +func (x *ClusterReplicationConfig) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_replication_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ClusterReplicationConfig.ProtoReflect.Descriptor instead. +func (*ClusterReplicationConfig) Descriptor() ([]byte, []int) { + return file_temporal_api_replication_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *ClusterReplicationConfig) GetClusterName() string { + if x != nil { + return x.ClusterName + } + return "" +} + +type NamespaceReplicationConfig struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ActiveClusterName string `protobuf:"bytes,1,opt,name=active_cluster_name,json=activeClusterName,proto3" json:"active_cluster_name,omitempty"` + Clusters []*ClusterReplicationConfig `protobuf:"bytes,2,rep,name=clusters,proto3" json:"clusters,omitempty"` + State v1.ReplicationState `protobuf:"varint,3,opt,name=state,proto3,enum=temporal.api.enums.v1.ReplicationState" json:"state,omitempty"` +} + +func (x *NamespaceReplicationConfig) Reset() { + *x = NamespaceReplicationConfig{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_replication_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NamespaceReplicationConfig) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NamespaceReplicationConfig) ProtoMessage() {} + +func (x *NamespaceReplicationConfig) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_replication_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NamespaceReplicationConfig.ProtoReflect.Descriptor instead. +func (*NamespaceReplicationConfig) Descriptor() ([]byte, []int) { + return file_temporal_api_replication_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *NamespaceReplicationConfig) GetActiveClusterName() string { + if x != nil { + return x.ActiveClusterName + } + return "" +} + +func (x *NamespaceReplicationConfig) GetClusters() []*ClusterReplicationConfig { + if x != nil { + return x.Clusters + } + return nil +} + +func (x *NamespaceReplicationConfig) GetState() v1.ReplicationState { + if x != nil { + return x.State + } + return v1.ReplicationState(0) +} + +// Represents a historical replication status of a Namespace +type FailoverStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Timestamp when the Cluster switched to the following failover_version + FailoverTime *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=failover_time,json=failoverTime,proto3" json:"failover_time,omitempty"` + FailoverVersion int64 `protobuf:"varint,2,opt,name=failover_version,json=failoverVersion,proto3" json:"failover_version,omitempty"` +} + +func (x *FailoverStatus) Reset() { + *x = FailoverStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_replication_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FailoverStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FailoverStatus) ProtoMessage() {} + +func (x *FailoverStatus) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_replication_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FailoverStatus.ProtoReflect.Descriptor instead. +func (*FailoverStatus) Descriptor() ([]byte, []int) { + return file_temporal_api_replication_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *FailoverStatus) GetFailoverTime() *timestamppb.Timestamp { + if x != nil { + return x.FailoverTime + } + return nil +} + +func (x *FailoverStatus) GetFailoverVersion() int64 { + if x != nil { + return x.FailoverVersion + } + return 0 +} + +var File_temporal_api_replication_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_replication_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x29, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x72, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x1b, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x41, 0x0a, 0x18, 0x43, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x25, 0x0a, 0x0c, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xea, 0x01, 0x0a, 0x1a, + 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x32, 0x0a, 0x13, 0x61, 0x63, 0x74, + 0x69, 0x76, 0x65, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x55, 0x0a, 0x08, + 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x72, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, + 0x65, 0x72, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x52, 0x08, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x41, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, + 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x84, 0x01, 0x0a, 0x0e, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x12, 0x43, 0x0a, 0x0d, 0x66, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0c, 0x66, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x54, 0x69, + 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x66, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, + 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0f, 0x66, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x42, 0xa2, 0x01, 0x0a, 0x1e, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, + 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x2d, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x3b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0xaa, 0x02, 0x1d, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, + 0x70, 0x69, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x56, + 0x31, 0xea, 0x02, 0x20, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, + 0x70, 0x69, 0x3a, 0x3a, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x3a, + 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_replication_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_replication_v1_message_proto_rawDescData = file_temporal_api_replication_v1_message_proto_rawDesc +) + +func file_temporal_api_replication_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_replication_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_replication_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_replication_v1_message_proto_rawDescData) + }) + return file_temporal_api_replication_v1_message_proto_rawDescData +} + +var file_temporal_api_replication_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 3) +var file_temporal_api_replication_v1_message_proto_goTypes = []any{ + (*ClusterReplicationConfig)(nil), // 0: temporal.api.replication.v1.ClusterReplicationConfig + (*NamespaceReplicationConfig)(nil), // 1: temporal.api.replication.v1.NamespaceReplicationConfig + (*FailoverStatus)(nil), // 2: temporal.api.replication.v1.FailoverStatus + (v1.ReplicationState)(0), // 3: temporal.api.enums.v1.ReplicationState + (*timestamppb.Timestamp)(nil), // 4: google.protobuf.Timestamp +} +var file_temporal_api_replication_v1_message_proto_depIdxs = []int32{ + 0, // 0: temporal.api.replication.v1.NamespaceReplicationConfig.clusters:type_name -> temporal.api.replication.v1.ClusterReplicationConfig + 3, // 1: temporal.api.replication.v1.NamespaceReplicationConfig.state:type_name -> temporal.api.enums.v1.ReplicationState + 4, // 2: temporal.api.replication.v1.FailoverStatus.failover_time:type_name -> google.protobuf.Timestamp + 3, // [3:3] is the sub-list for method output_type + 3, // [3:3] is the sub-list for method input_type + 3, // [3:3] is the sub-list for extension type_name + 3, // [3:3] is the sub-list for extension extendee + 0, // [0:3] is the sub-list for field type_name +} + +func init() { file_temporal_api_replication_v1_message_proto_init() } +func file_temporal_api_replication_v1_message_proto_init() { + if File_temporal_api_replication_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_replication_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*ClusterReplicationConfig); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_replication_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*NamespaceReplicationConfig); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_replication_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*FailoverStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_replication_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 3, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_replication_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_replication_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_replication_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_replication_v1_message_proto = out.File + file_temporal_api_replication_v1_message_proto_rawDesc = nil + file_temporal_api_replication_v1_message_proto_goTypes = nil + file_temporal_api_replication_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/schedule/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/schedule/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..b1d6e96111d --- /dev/null +++ b/vendor/go.temporal.io/api/schedule/v1/message.go-helpers.pb.go @@ -0,0 +1,620 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package schedule + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type CalendarSpec to the protobuf v3 wire format +func (val *CalendarSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CalendarSpec from the protobuf v3 wire format +func (val *CalendarSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CalendarSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CalendarSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CalendarSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CalendarSpec + switch t := that.(type) { + case *CalendarSpec: + that1 = t + case CalendarSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Range to the protobuf v3 wire format +func (val *Range) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Range from the protobuf v3 wire format +func (val *Range) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Range) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Range values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Range) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Range + switch t := that.(type) { + case *Range: + that1 = t + case Range: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StructuredCalendarSpec to the protobuf v3 wire format +func (val *StructuredCalendarSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StructuredCalendarSpec from the protobuf v3 wire format +func (val *StructuredCalendarSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StructuredCalendarSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StructuredCalendarSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StructuredCalendarSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StructuredCalendarSpec + switch t := that.(type) { + case *StructuredCalendarSpec: + that1 = t + case StructuredCalendarSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type IntervalSpec to the protobuf v3 wire format +func (val *IntervalSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type IntervalSpec from the protobuf v3 wire format +func (val *IntervalSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *IntervalSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two IntervalSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *IntervalSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *IntervalSpec + switch t := that.(type) { + case *IntervalSpec: + that1 = t + case IntervalSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ScheduleSpec to the protobuf v3 wire format +func (val *ScheduleSpec) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ScheduleSpec from the protobuf v3 wire format +func (val *ScheduleSpec) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ScheduleSpec) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ScheduleSpec values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ScheduleSpec) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ScheduleSpec + switch t := that.(type) { + case *ScheduleSpec: + that1 = t + case ScheduleSpec: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SchedulePolicies to the protobuf v3 wire format +func (val *SchedulePolicies) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SchedulePolicies from the protobuf v3 wire format +func (val *SchedulePolicies) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SchedulePolicies) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SchedulePolicies values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SchedulePolicies) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SchedulePolicies + switch t := that.(type) { + case *SchedulePolicies: + that1 = t + case SchedulePolicies: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ScheduleAction to the protobuf v3 wire format +func (val *ScheduleAction) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ScheduleAction from the protobuf v3 wire format +func (val *ScheduleAction) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ScheduleAction) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ScheduleAction values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ScheduleAction) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ScheduleAction + switch t := that.(type) { + case *ScheduleAction: + that1 = t + case ScheduleAction: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ScheduleActionResult to the protobuf v3 wire format +func (val *ScheduleActionResult) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ScheduleActionResult from the protobuf v3 wire format +func (val *ScheduleActionResult) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ScheduleActionResult) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ScheduleActionResult values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ScheduleActionResult) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ScheduleActionResult + switch t := that.(type) { + case *ScheduleActionResult: + that1 = t + case ScheduleActionResult: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ScheduleState to the protobuf v3 wire format +func (val *ScheduleState) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ScheduleState from the protobuf v3 wire format +func (val *ScheduleState) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ScheduleState) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ScheduleState values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ScheduleState) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ScheduleState + switch t := that.(type) { + case *ScheduleState: + that1 = t + case ScheduleState: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TriggerImmediatelyRequest to the protobuf v3 wire format +func (val *TriggerImmediatelyRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TriggerImmediatelyRequest from the protobuf v3 wire format +func (val *TriggerImmediatelyRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TriggerImmediatelyRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TriggerImmediatelyRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TriggerImmediatelyRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TriggerImmediatelyRequest + switch t := that.(type) { + case *TriggerImmediatelyRequest: + that1 = t + case TriggerImmediatelyRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type BackfillRequest to the protobuf v3 wire format +func (val *BackfillRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type BackfillRequest from the protobuf v3 wire format +func (val *BackfillRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *BackfillRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two BackfillRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *BackfillRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *BackfillRequest + switch t := that.(type) { + case *BackfillRequest: + that1 = t + case BackfillRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SchedulePatch to the protobuf v3 wire format +func (val *SchedulePatch) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SchedulePatch from the protobuf v3 wire format +func (val *SchedulePatch) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SchedulePatch) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SchedulePatch values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SchedulePatch) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SchedulePatch + switch t := that.(type) { + case *SchedulePatch: + that1 = t + case SchedulePatch: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ScheduleInfo to the protobuf v3 wire format +func (val *ScheduleInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ScheduleInfo from the protobuf v3 wire format +func (val *ScheduleInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ScheduleInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ScheduleInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ScheduleInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ScheduleInfo + switch t := that.(type) { + case *ScheduleInfo: + that1 = t + case ScheduleInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Schedule to the protobuf v3 wire format +func (val *Schedule) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Schedule from the protobuf v3 wire format +func (val *Schedule) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Schedule) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Schedule values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Schedule) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Schedule + switch t := that.(type) { + case *Schedule: + that1 = t + case Schedule: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ScheduleListInfo to the protobuf v3 wire format +func (val *ScheduleListInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ScheduleListInfo from the protobuf v3 wire format +func (val *ScheduleListInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ScheduleListInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ScheduleListInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ScheduleListInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ScheduleListInfo + switch t := that.(type) { + case *ScheduleListInfo: + that1 = t + case ScheduleListInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ScheduleListEntry to the protobuf v3 wire format +func (val *ScheduleListEntry) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ScheduleListEntry from the protobuf v3 wire format +func (val *ScheduleListEntry) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ScheduleListEntry) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ScheduleListEntry values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ScheduleListEntry) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ScheduleListEntry + switch t := that.(type) { + case *ScheduleListEntry: + that1 = t + case ScheduleListEntry: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/schedule/v1/message.pb.go b/vendor/go.temporal.io/api/schedule/v1/message.pb.go new file mode 100644 index 00000000000..967513156f1 --- /dev/null +++ b/vendor/go.temporal.io/api/schedule/v1/message.pb.go @@ -0,0 +1,2156 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// (-- api-linter: core::0203::optional=disabled +// aip.dev/not-precedent: field_behavior annotation not available in our gogo fork --) +// (-- api-linter: core::0203::input-only=disabled +// aip.dev/not-precedent: field_behavior annotation not available in our gogo fork --) + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/schedule/v1/message.proto + +package schedule + +import ( + reflect "reflect" + sync "sync" + + v12 "go.temporal.io/api/common/v1" + v1 "go.temporal.io/api/enums/v1" + v11 "go.temporal.io/api/workflow/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + durationpb "google.golang.org/protobuf/types/known/durationpb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// CalendarSpec describes an event specification relative to the calendar, +// similar to a traditional cron specification, but with labeled fields. Each +// field can be one of: +// +// *: matches always +// x: matches when the field equals x +// x/y : matches when the field equals x+n*y where n is an integer +// x-z: matches when the field is between x and z inclusive +// w,x,y,...: matches when the field is one of the listed values +// +// Each x, y, z, ... is either a decimal integer, or a month or day of week name +// or abbreviation (in the appropriate fields). +// A timestamp matches if all fields match. +// Note that fields have different default values, for convenience. +// Note that the special case that some cron implementations have for treating +// day_of_month and day_of_week as "or" instead of "and" when both are set is +// not implemented. +// day_of_week can accept 0 or 7 as Sunday +// CalendarSpec gets compiled into StructuredCalendarSpec, which is what will be +// returned if you describe the schedule. +type CalendarSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Expression to match seconds. Default: 0 + Second string `protobuf:"bytes,1,opt,name=second,proto3" json:"second,omitempty"` + // Expression to match minutes. Default: 0 + Minute string `protobuf:"bytes,2,opt,name=minute,proto3" json:"minute,omitempty"` + // Expression to match hours. Default: 0 + Hour string `protobuf:"bytes,3,opt,name=hour,proto3" json:"hour,omitempty"` + // Expression to match days of the month. Default: * + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: standard name of field --) + DayOfMonth string `protobuf:"bytes,4,opt,name=day_of_month,json=dayOfMonth,proto3" json:"day_of_month,omitempty"` + // Expression to match months. Default: * + Month string `protobuf:"bytes,5,opt,name=month,proto3" json:"month,omitempty"` + // Expression to match years. Default: * + Year string `protobuf:"bytes,6,opt,name=year,proto3" json:"year,omitempty"` + // Expression to match days of the week. Default: * + DayOfWeek string `protobuf:"bytes,7,opt,name=day_of_week,json=dayOfWeek,proto3" json:"day_of_week,omitempty"` + // Free-form comment describing the intention of this spec. + Comment string `protobuf:"bytes,8,opt,name=comment,proto3" json:"comment,omitempty"` +} + +func (x *CalendarSpec) Reset() { + *x = CalendarSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CalendarSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CalendarSpec) ProtoMessage() {} + +func (x *CalendarSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CalendarSpec.ProtoReflect.Descriptor instead. +func (*CalendarSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *CalendarSpec) GetSecond() string { + if x != nil { + return x.Second + } + return "" +} + +func (x *CalendarSpec) GetMinute() string { + if x != nil { + return x.Minute + } + return "" +} + +func (x *CalendarSpec) GetHour() string { + if x != nil { + return x.Hour + } + return "" +} + +func (x *CalendarSpec) GetDayOfMonth() string { + if x != nil { + return x.DayOfMonth + } + return "" +} + +func (x *CalendarSpec) GetMonth() string { + if x != nil { + return x.Month + } + return "" +} + +func (x *CalendarSpec) GetYear() string { + if x != nil { + return x.Year + } + return "" +} + +func (x *CalendarSpec) GetDayOfWeek() string { + if x != nil { + return x.DayOfWeek + } + return "" +} + +func (x *CalendarSpec) GetComment() string { + if x != nil { + return x.Comment + } + return "" +} + +// Range represents a set of integer values, used to match fields of a calendar +// time in StructuredCalendarSpec. If end < start, then end is interpreted as +// equal to start. This means you can use a Range with start set to a value, and +// end and step unset (defaulting to 0) to represent a single value. +type Range struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Start of range (inclusive). + Start int32 `protobuf:"varint,1,opt,name=start,proto3" json:"start,omitempty"` + // End of range (inclusive). + End int32 `protobuf:"varint,2,opt,name=end,proto3" json:"end,omitempty"` + // Step (optional, default 1). + Step int32 `protobuf:"varint,3,opt,name=step,proto3" json:"step,omitempty"` +} + +func (x *Range) Reset() { + *x = Range{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Range) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Range) ProtoMessage() {} + +func (x *Range) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Range.ProtoReflect.Descriptor instead. +func (*Range) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *Range) GetStart() int32 { + if x != nil { + return x.Start + } + return 0 +} + +func (x *Range) GetEnd() int32 { + if x != nil { + return x.End + } + return 0 +} + +func (x *Range) GetStep() int32 { + if x != nil { + return x.Step + } + return 0 +} + +// StructuredCalendarSpec describes an event specification relative to the +// calendar, in a form that's easy to work with programmatically. Each field can +// be one or more ranges. +// A timestamp matches if at least one range of each field matches the +// corresponding fields of the timestamp, except for year: if year is missing, +// that means all years match. For all fields besides year, at least one Range +// must be present to match anything. +// TODO: add relative-to-end-of-month +// TODO: add nth day-of-week in month +type StructuredCalendarSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Match seconds (0-59) + Second []*Range `protobuf:"bytes,1,rep,name=second,proto3" json:"second,omitempty"` + // Match minutes (0-59) + Minute []*Range `protobuf:"bytes,2,rep,name=minute,proto3" json:"minute,omitempty"` + // Match hours (0-23) + Hour []*Range `protobuf:"bytes,3,rep,name=hour,proto3" json:"hour,omitempty"` + // Match days of the month (1-31) + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: standard name of field --) + DayOfMonth []*Range `protobuf:"bytes,4,rep,name=day_of_month,json=dayOfMonth,proto3" json:"day_of_month,omitempty"` + // Match months (1-12) + Month []*Range `protobuf:"bytes,5,rep,name=month,proto3" json:"month,omitempty"` + // Match years. + Year []*Range `protobuf:"bytes,6,rep,name=year,proto3" json:"year,omitempty"` + // Match days of the week (0-6; 0 is Sunday). + DayOfWeek []*Range `protobuf:"bytes,7,rep,name=day_of_week,json=dayOfWeek,proto3" json:"day_of_week,omitempty"` + // Free-form comment describing the intention of this spec. + Comment string `protobuf:"bytes,8,opt,name=comment,proto3" json:"comment,omitempty"` +} + +func (x *StructuredCalendarSpec) Reset() { + *x = StructuredCalendarSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StructuredCalendarSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StructuredCalendarSpec) ProtoMessage() {} + +func (x *StructuredCalendarSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StructuredCalendarSpec.ProtoReflect.Descriptor instead. +func (*StructuredCalendarSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *StructuredCalendarSpec) GetSecond() []*Range { + if x != nil { + return x.Second + } + return nil +} + +func (x *StructuredCalendarSpec) GetMinute() []*Range { + if x != nil { + return x.Minute + } + return nil +} + +func (x *StructuredCalendarSpec) GetHour() []*Range { + if x != nil { + return x.Hour + } + return nil +} + +func (x *StructuredCalendarSpec) GetDayOfMonth() []*Range { + if x != nil { + return x.DayOfMonth + } + return nil +} + +func (x *StructuredCalendarSpec) GetMonth() []*Range { + if x != nil { + return x.Month + } + return nil +} + +func (x *StructuredCalendarSpec) GetYear() []*Range { + if x != nil { + return x.Year + } + return nil +} + +func (x *StructuredCalendarSpec) GetDayOfWeek() []*Range { + if x != nil { + return x.DayOfWeek + } + return nil +} + +func (x *StructuredCalendarSpec) GetComment() string { + if x != nil { + return x.Comment + } + return "" +} + +// IntervalSpec matches times that can be expressed as: +// epoch + n * interval + phase +// where n is an integer. +// phase defaults to zero if missing. interval is required. +// Both interval and phase must be non-negative and are truncated to the nearest +// second before any calculations. +// For example, an interval of 1 hour with phase of zero would match every hour, +// on the hour. The same interval but a phase of 19 minutes would match every +// xx:19:00. An interval of 28 days with phase zero would match +// 2022-02-17T00:00:00Z (among other times). The same interval with a phase of 3 +// days, 5 hours, and 23 minutes would match 2022-02-20T05:23:00Z instead. +type IntervalSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Interval *durationpb.Duration `protobuf:"bytes,1,opt,name=interval,proto3" json:"interval,omitempty"` + Phase *durationpb.Duration `protobuf:"bytes,2,opt,name=phase,proto3" json:"phase,omitempty"` +} + +func (x *IntervalSpec) Reset() { + *x = IntervalSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *IntervalSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IntervalSpec) ProtoMessage() {} + +func (x *IntervalSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use IntervalSpec.ProtoReflect.Descriptor instead. +func (*IntervalSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *IntervalSpec) GetInterval() *durationpb.Duration { + if x != nil { + return x.Interval + } + return nil +} + +func (x *IntervalSpec) GetPhase() *durationpb.Duration { + if x != nil { + return x.Phase + } + return nil +} + +// ScheduleSpec is a complete description of a set of absolute timestamps +// (possibly infinite) that an action should occur at. The meaning of a +// ScheduleSpec depends only on its contents and never changes, except that the +// definition of a time zone can change over time (most commonly, when daylight +// saving time policy changes for an area). To create a totally self-contained +// ScheduleSpec, use UTC or include timezone_data. +// +// For input, you can provide zero or more of: structured_calendar, calendar, +// cron_string, interval, and exclude_structured_calendar, and all of them will +// be used (the schedule will take action at the union of all of their times, +// minus the ones that match exclude_structured_calendar). +// +// On input, calendar and cron_string fields will be compiled into +// structured_calendar (and maybe interval and timezone_name), so if you +// Describe a schedule, you'll see only structured_calendar, interval, etc. +// +// If a spec has no matching times after the current time, then the schedule +// will be subject to automatic deletion (after several days). +type ScheduleSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Calendar-based specifications of times. + StructuredCalendar []*StructuredCalendarSpec `protobuf:"bytes,7,rep,name=structured_calendar,json=structuredCalendar,proto3" json:"structured_calendar,omitempty"` + // cron_string holds a traditional cron specification as a string. It + // accepts 5, 6, or 7 fields, separated by spaces, and interprets them the + // same way as CalendarSpec. + // 5 fields: minute, hour, day_of_month, month, day_of_week + // 6 fields: minute, hour, day_of_month, month, day_of_week, year + // 7 fields: second, minute, hour, day_of_month, month, day_of_week, year + // If year is not given, it defaults to *. If second is not given, it + // defaults to 0. + // Shorthands @yearly, @monthly, @weekly, @daily, and @hourly are also + // accepted instead of the 5-7 time fields. + // Optionally, the string can be preceded by CRON_TZ= or + // TZ=, which will get copied to timezone_name. (There must + // not also be a timezone_name present.) + // Optionally "#" followed by a comment can appear at the end of the string. + // Note that the special case that some cron implementations have for + // treating day_of_month and day_of_week as "or" instead of "and" when both + // are set is not implemented. + // @every [/] is accepted and gets compiled into an + // IntervalSpec instead. and should be a decimal integer + // with a unit suffix s, m, h, or d. + CronString []string `protobuf:"bytes,8,rep,name=cron_string,json=cronString,proto3" json:"cron_string,omitempty"` + // Calendar-based specifications of times. + Calendar []*CalendarSpec `protobuf:"bytes,1,rep,name=calendar,proto3" json:"calendar,omitempty"` + // Interval-based specifications of times. + Interval []*IntervalSpec `protobuf:"bytes,2,rep,name=interval,proto3" json:"interval,omitempty"` + // Any timestamps matching any of exclude_* will be skipped. + // + // Deprecated: Marked as deprecated in temporal/api/schedule/v1/message.proto. + ExcludeCalendar []*CalendarSpec `protobuf:"bytes,3,rep,name=exclude_calendar,json=excludeCalendar,proto3" json:"exclude_calendar,omitempty"` // use exclude_structured_calendar + ExcludeStructuredCalendar []*StructuredCalendarSpec `protobuf:"bytes,9,rep,name=exclude_structured_calendar,json=excludeStructuredCalendar,proto3" json:"exclude_structured_calendar,omitempty"` + // If start_time is set, any timestamps before start_time will be skipped. + // (Together, start_time and end_time make an inclusive interval.) + StartTime *timestamppb.Timestamp `protobuf:"bytes,4,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + // If end_time is set, any timestamps after end_time will be skipped. + EndTime *timestamppb.Timestamp `protobuf:"bytes,5,opt,name=end_time,json=endTime,proto3" json:"end_time,omitempty"` + // All timestamps will be incremented by a random value from 0 to this + // amount of jitter. Default: 0 + Jitter *durationpb.Duration `protobuf:"bytes,6,opt,name=jitter,proto3" json:"jitter,omitempty"` + // Time zone to interpret all calendar-based specs in. + // + // If unset, defaults to UTC. We recommend using UTC for your application if + // at all possible, to avoid various surprising properties of time zones. + // + // Time zones may be provided by name, corresponding to names in the IANA + // time zone database (see https://www.iana.org/time-zones). The definition + // will be loaded by the Temporal server from the environment it runs in. + // + // If your application requires more control over the time zone definition + // used, it may pass in a complete definition in the form of a TZif file + // from the time zone database. If present, this will be used instead of + // loading anything from the environment. You are then responsible for + // updating timezone_data when the definition changes. + // + // Calendar spec matching is based on literal matching of the clock time + // with no special handling of DST: if you write a calendar spec that fires + // at 2:30am and specify a time zone that follows DST, that action will not + // be triggered on the day that has no 2:30am. Similarly, an action that + // fires at 1:30am will be triggered twice on the day that has two 1:30s. + // + // Also note that no actions are taken on leap-seconds (e.g. 23:59:60 UTC). + TimezoneName string `protobuf:"bytes,10,opt,name=timezone_name,json=timezoneName,proto3" json:"timezone_name,omitempty"` + TimezoneData []byte `protobuf:"bytes,11,opt,name=timezone_data,json=timezoneData,proto3" json:"timezone_data,omitempty"` +} + +func (x *ScheduleSpec) Reset() { + *x = ScheduleSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ScheduleSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ScheduleSpec) ProtoMessage() {} + +func (x *ScheduleSpec) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ScheduleSpec.ProtoReflect.Descriptor instead. +func (*ScheduleSpec) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *ScheduleSpec) GetStructuredCalendar() []*StructuredCalendarSpec { + if x != nil { + return x.StructuredCalendar + } + return nil +} + +func (x *ScheduleSpec) GetCronString() []string { + if x != nil { + return x.CronString + } + return nil +} + +func (x *ScheduleSpec) GetCalendar() []*CalendarSpec { + if x != nil { + return x.Calendar + } + return nil +} + +func (x *ScheduleSpec) GetInterval() []*IntervalSpec { + if x != nil { + return x.Interval + } + return nil +} + +// Deprecated: Marked as deprecated in temporal/api/schedule/v1/message.proto. +func (x *ScheduleSpec) GetExcludeCalendar() []*CalendarSpec { + if x != nil { + return x.ExcludeCalendar + } + return nil +} + +func (x *ScheduleSpec) GetExcludeStructuredCalendar() []*StructuredCalendarSpec { + if x != nil { + return x.ExcludeStructuredCalendar + } + return nil +} + +func (x *ScheduleSpec) GetStartTime() *timestamppb.Timestamp { + if x != nil { + return x.StartTime + } + return nil +} + +func (x *ScheduleSpec) GetEndTime() *timestamppb.Timestamp { + if x != nil { + return x.EndTime + } + return nil +} + +func (x *ScheduleSpec) GetJitter() *durationpb.Duration { + if x != nil { + return x.Jitter + } + return nil +} + +func (x *ScheduleSpec) GetTimezoneName() string { + if x != nil { + return x.TimezoneName + } + return "" +} + +func (x *ScheduleSpec) GetTimezoneData() []byte { + if x != nil { + return x.TimezoneData + } + return nil +} + +type SchedulePolicies struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Policy for overlaps. + // Note that this can be changed after a schedule has taken some actions, + // and some changes might produce unintuitive results. In general, the later + // policy overrides the earlier policy. + OverlapPolicy v1.ScheduleOverlapPolicy `protobuf:"varint,1,opt,name=overlap_policy,json=overlapPolicy,proto3,enum=temporal.api.enums.v1.ScheduleOverlapPolicy" json:"overlap_policy,omitempty"` + // Policy for catchups: + // If the Temporal server misses an action due to one or more components + // being down, and comes back up, the action will be run if the scheduled + // time is within this window from the current time. + // This value defaults to one year, and can't be less than 10 seconds. + CatchupWindow *durationpb.Duration `protobuf:"bytes,2,opt,name=catchup_window,json=catchupWindow,proto3" json:"catchup_window,omitempty"` + // If true, and a workflow run fails or times out, turn on "paused". + // This applies after retry policies: the full chain of retries must fail to + // trigger a pause here. + PauseOnFailure bool `protobuf:"varint,3,opt,name=pause_on_failure,json=pauseOnFailure,proto3" json:"pause_on_failure,omitempty"` + // If true, and the action would start a workflow, a timestamp will not be + // appended to the scheduled workflow id. + KeepOriginalWorkflowId bool `protobuf:"varint,4,opt,name=keep_original_workflow_id,json=keepOriginalWorkflowId,proto3" json:"keep_original_workflow_id,omitempty"` +} + +func (x *SchedulePolicies) Reset() { + *x = SchedulePolicies{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SchedulePolicies) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SchedulePolicies) ProtoMessage() {} + +func (x *SchedulePolicies) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SchedulePolicies.ProtoReflect.Descriptor instead. +func (*SchedulePolicies) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{5} +} + +func (x *SchedulePolicies) GetOverlapPolicy() v1.ScheduleOverlapPolicy { + if x != nil { + return x.OverlapPolicy + } + return v1.ScheduleOverlapPolicy(0) +} + +func (x *SchedulePolicies) GetCatchupWindow() *durationpb.Duration { + if x != nil { + return x.CatchupWindow + } + return nil +} + +func (x *SchedulePolicies) GetPauseOnFailure() bool { + if x != nil { + return x.PauseOnFailure + } + return false +} + +func (x *SchedulePolicies) GetKeepOriginalWorkflowId() bool { + if x != nil { + return x.KeepOriginalWorkflowId + } + return false +} + +type ScheduleAction struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Action: + // + // *ScheduleAction_StartWorkflow + Action isScheduleAction_Action `protobuf_oneof:"action"` +} + +func (x *ScheduleAction) Reset() { + *x = ScheduleAction{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ScheduleAction) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ScheduleAction) ProtoMessage() {} + +func (x *ScheduleAction) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ScheduleAction.ProtoReflect.Descriptor instead. +func (*ScheduleAction) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{6} +} + +func (m *ScheduleAction) GetAction() isScheduleAction_Action { + if m != nil { + return m.Action + } + return nil +} + +func (x *ScheduleAction) GetStartWorkflow() *v11.NewWorkflowExecutionInfo { + if x, ok := x.GetAction().(*ScheduleAction_StartWorkflow); ok { + return x.StartWorkflow + } + return nil +} + +type isScheduleAction_Action interface { + isScheduleAction_Action() +} + +type ScheduleAction_StartWorkflow struct { + // All fields of NewWorkflowExecutionInfo are valid except for: + // - workflow_id_reuse_policy + // - cron_schedule + // The workflow id of the started workflow may not match this exactly, + // it may have a timestamp appended for uniqueness. + StartWorkflow *v11.NewWorkflowExecutionInfo `protobuf:"bytes,1,opt,name=start_workflow,json=startWorkflow,proto3,oneof"` +} + +func (*ScheduleAction_StartWorkflow) isScheduleAction_Action() {} + +type ScheduleActionResult struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Time that the action was taken (according to the schedule, including jitter). + ScheduleTime *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=schedule_time,json=scheduleTime,proto3" json:"schedule_time,omitempty"` + // Time that the action was taken (real time). + ActualTime *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=actual_time,json=actualTime,proto3" json:"actual_time,omitempty"` + // If action was start_workflow: + StartWorkflowResult *v12.WorkflowExecution `protobuf:"bytes,11,opt,name=start_workflow_result,json=startWorkflowResult,proto3" json:"start_workflow_result,omitempty"` +} + +func (x *ScheduleActionResult) Reset() { + *x = ScheduleActionResult{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ScheduleActionResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ScheduleActionResult) ProtoMessage() {} + +func (x *ScheduleActionResult) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ScheduleActionResult.ProtoReflect.Descriptor instead. +func (*ScheduleActionResult) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{7} +} + +func (x *ScheduleActionResult) GetScheduleTime() *timestamppb.Timestamp { + if x != nil { + return x.ScheduleTime + } + return nil +} + +func (x *ScheduleActionResult) GetActualTime() *timestamppb.Timestamp { + if x != nil { + return x.ActualTime + } + return nil +} + +func (x *ScheduleActionResult) GetStartWorkflowResult() *v12.WorkflowExecution { + if x != nil { + return x.StartWorkflowResult + } + return nil +} + +type ScheduleState struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Informative human-readable message with contextual notes, e.g. the reason + // a schedule is paused. The system may overwrite this message on certain + // conditions, e.g. when pause-on-failure happens. + Notes string `protobuf:"bytes,1,opt,name=notes,proto3" json:"notes,omitempty"` + // If true, do not take any actions based on the schedule spec. + Paused bool `protobuf:"varint,2,opt,name=paused,proto3" json:"paused,omitempty"` + // If limited_actions is true, decrement remaining_actions after each + // action, and do not take any more scheduled actions if remaining_actions + // is zero. Actions may still be taken by explicit request (i.e. trigger + // immediately or backfill). Skipped actions (due to overlap policy) do not + // count against remaining actions. + // If a schedule has no more remaining actions, then the schedule will be + // subject to automatic deletion (after several days). + LimitedActions bool `protobuf:"varint,3,opt,name=limited_actions,json=limitedActions,proto3" json:"limited_actions,omitempty"` + RemainingActions int64 `protobuf:"varint,4,opt,name=remaining_actions,json=remainingActions,proto3" json:"remaining_actions,omitempty"` +} + +func (x *ScheduleState) Reset() { + *x = ScheduleState{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ScheduleState) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ScheduleState) ProtoMessage() {} + +func (x *ScheduleState) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ScheduleState.ProtoReflect.Descriptor instead. +func (*ScheduleState) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{8} +} + +func (x *ScheduleState) GetNotes() string { + if x != nil { + return x.Notes + } + return "" +} + +func (x *ScheduleState) GetPaused() bool { + if x != nil { + return x.Paused + } + return false +} + +func (x *ScheduleState) GetLimitedActions() bool { + if x != nil { + return x.LimitedActions + } + return false +} + +func (x *ScheduleState) GetRemainingActions() int64 { + if x != nil { + return x.RemainingActions + } + return 0 +} + +type TriggerImmediatelyRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // If set, override overlap policy for this one request. + OverlapPolicy v1.ScheduleOverlapPolicy `protobuf:"varint,1,opt,name=overlap_policy,json=overlapPolicy,proto3,enum=temporal.api.enums.v1.ScheduleOverlapPolicy" json:"overlap_policy,omitempty"` +} + +func (x *TriggerImmediatelyRequest) Reset() { + *x = TriggerImmediatelyRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TriggerImmediatelyRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TriggerImmediatelyRequest) ProtoMessage() {} + +func (x *TriggerImmediatelyRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TriggerImmediatelyRequest.ProtoReflect.Descriptor instead. +func (*TriggerImmediatelyRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{9} +} + +func (x *TriggerImmediatelyRequest) GetOverlapPolicy() v1.ScheduleOverlapPolicy { + if x != nil { + return x.OverlapPolicy + } + return v1.ScheduleOverlapPolicy(0) +} + +type BackfillRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Time range to evaluate schedule in. Currently, this time range is + // exclusive on start_time and inclusive on end_time. (This is admittedly + // counterintuitive and it may change in the future, so to be safe, use a + // start time strictly before a scheduled time.) Also note that an action + // nominally scheduled in the interval but with jitter that pushes it after + // end_time will not be included. + StartTime *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + EndTime *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=end_time,json=endTime,proto3" json:"end_time,omitempty"` + // If set, override overlap policy for this request. + OverlapPolicy v1.ScheduleOverlapPolicy `protobuf:"varint,3,opt,name=overlap_policy,json=overlapPolicy,proto3,enum=temporal.api.enums.v1.ScheduleOverlapPolicy" json:"overlap_policy,omitempty"` +} + +func (x *BackfillRequest) Reset() { + *x = BackfillRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BackfillRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BackfillRequest) ProtoMessage() {} + +func (x *BackfillRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BackfillRequest.ProtoReflect.Descriptor instead. +func (*BackfillRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{10} +} + +func (x *BackfillRequest) GetStartTime() *timestamppb.Timestamp { + if x != nil { + return x.StartTime + } + return nil +} + +func (x *BackfillRequest) GetEndTime() *timestamppb.Timestamp { + if x != nil { + return x.EndTime + } + return nil +} + +func (x *BackfillRequest) GetOverlapPolicy() v1.ScheduleOverlapPolicy { + if x != nil { + return x.OverlapPolicy + } + return v1.ScheduleOverlapPolicy(0) +} + +type SchedulePatch struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // If set, trigger one action immediately. + TriggerImmediately *TriggerImmediatelyRequest `protobuf:"bytes,1,opt,name=trigger_immediately,json=triggerImmediately,proto3" json:"trigger_immediately,omitempty"` + // If set, runs though the specified time period(s) and takes actions as if that time + // passed by right now, all at once. The overlap policy can be overridden for the + // scope of the backfill. + BackfillRequest []*BackfillRequest `protobuf:"bytes,2,rep,name=backfill_request,json=backfillRequest,proto3" json:"backfill_request,omitempty"` + // If set, change the state to paused or unpaused (respectively) and set the + // notes field to the value of the string. + Pause string `protobuf:"bytes,3,opt,name=pause,proto3" json:"pause,omitempty"` + Unpause string `protobuf:"bytes,4,opt,name=unpause,proto3" json:"unpause,omitempty"` +} + +func (x *SchedulePatch) Reset() { + *x = SchedulePatch{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SchedulePatch) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SchedulePatch) ProtoMessage() {} + +func (x *SchedulePatch) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SchedulePatch.ProtoReflect.Descriptor instead. +func (*SchedulePatch) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{11} +} + +func (x *SchedulePatch) GetTriggerImmediately() *TriggerImmediatelyRequest { + if x != nil { + return x.TriggerImmediately + } + return nil +} + +func (x *SchedulePatch) GetBackfillRequest() []*BackfillRequest { + if x != nil { + return x.BackfillRequest + } + return nil +} + +func (x *SchedulePatch) GetPause() string { + if x != nil { + return x.Pause + } + return "" +} + +func (x *SchedulePatch) GetUnpause() string { + if x != nil { + return x.Unpause + } + return "" +} + +type ScheduleInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Number of actions taken so far. + ActionCount int64 `protobuf:"varint,1,opt,name=action_count,json=actionCount,proto3" json:"action_count,omitempty"` + // Number of times a scheduled action was skipped due to missing the catchup window. + MissedCatchupWindow int64 `protobuf:"varint,2,opt,name=missed_catchup_window,json=missedCatchupWindow,proto3" json:"missed_catchup_window,omitempty"` + // Number of skipped actions due to overlap. + OverlapSkipped int64 `protobuf:"varint,3,opt,name=overlap_skipped,json=overlapSkipped,proto3" json:"overlap_skipped,omitempty"` + // Number of dropped actions due to buffer limit. + BufferDropped int64 `protobuf:"varint,10,opt,name=buffer_dropped,json=bufferDropped,proto3" json:"buffer_dropped,omitempty"` + // Number of actions in the buffer. The buffer holds the actions that cannot + // be immediately triggered (due to the overlap policy). These actions can be a result of + // the normal schedule or a backfill. + BufferSize int64 `protobuf:"varint,11,opt,name=buffer_size,json=bufferSize,proto3" json:"buffer_size,omitempty"` + // Currently-running workflows started by this schedule. (There might be + // more than one if the overlap policy allows overlaps.) + // Note that the run_ids in here are the original execution run ids as + // started by the schedule. If the workflows retried, did continue-as-new, + // or were reset, they might still be running but with a different run_id. + RunningWorkflows []*v12.WorkflowExecution `protobuf:"bytes,9,rep,name=running_workflows,json=runningWorkflows,proto3" json:"running_workflows,omitempty"` + // Most recent ten actual action times (including manual triggers). + RecentActions []*ScheduleActionResult `protobuf:"bytes,4,rep,name=recent_actions,json=recentActions,proto3" json:"recent_actions,omitempty"` + // Next ten scheduled action times. + FutureActionTimes []*timestamppb.Timestamp `protobuf:"bytes,5,rep,name=future_action_times,json=futureActionTimes,proto3" json:"future_action_times,omitempty"` + // Timestamps of schedule creation and last update. + CreateTime *timestamppb.Timestamp `protobuf:"bytes,6,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` + UpdateTime *timestamppb.Timestamp `protobuf:"bytes,7,opt,name=update_time,json=updateTime,proto3" json:"update_time,omitempty"` + // Deprecated: Marked as deprecated in temporal/api/schedule/v1/message.proto. + InvalidScheduleError string `protobuf:"bytes,8,opt,name=invalid_schedule_error,json=invalidScheduleError,proto3" json:"invalid_schedule_error,omitempty"` +} + +func (x *ScheduleInfo) Reset() { + *x = ScheduleInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ScheduleInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ScheduleInfo) ProtoMessage() {} + +func (x *ScheduleInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ScheduleInfo.ProtoReflect.Descriptor instead. +func (*ScheduleInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{12} +} + +func (x *ScheduleInfo) GetActionCount() int64 { + if x != nil { + return x.ActionCount + } + return 0 +} + +func (x *ScheduleInfo) GetMissedCatchupWindow() int64 { + if x != nil { + return x.MissedCatchupWindow + } + return 0 +} + +func (x *ScheduleInfo) GetOverlapSkipped() int64 { + if x != nil { + return x.OverlapSkipped + } + return 0 +} + +func (x *ScheduleInfo) GetBufferDropped() int64 { + if x != nil { + return x.BufferDropped + } + return 0 +} + +func (x *ScheduleInfo) GetBufferSize() int64 { + if x != nil { + return x.BufferSize + } + return 0 +} + +func (x *ScheduleInfo) GetRunningWorkflows() []*v12.WorkflowExecution { + if x != nil { + return x.RunningWorkflows + } + return nil +} + +func (x *ScheduleInfo) GetRecentActions() []*ScheduleActionResult { + if x != nil { + return x.RecentActions + } + return nil +} + +func (x *ScheduleInfo) GetFutureActionTimes() []*timestamppb.Timestamp { + if x != nil { + return x.FutureActionTimes + } + return nil +} + +func (x *ScheduleInfo) GetCreateTime() *timestamppb.Timestamp { + if x != nil { + return x.CreateTime + } + return nil +} + +func (x *ScheduleInfo) GetUpdateTime() *timestamppb.Timestamp { + if x != nil { + return x.UpdateTime + } + return nil +} + +// Deprecated: Marked as deprecated in temporal/api/schedule/v1/message.proto. +func (x *ScheduleInfo) GetInvalidScheduleError() string { + if x != nil { + return x.InvalidScheduleError + } + return "" +} + +type Schedule struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Spec *ScheduleSpec `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"` + Action *ScheduleAction `protobuf:"bytes,2,opt,name=action,proto3" json:"action,omitempty"` + Policies *SchedulePolicies `protobuf:"bytes,3,opt,name=policies,proto3" json:"policies,omitempty"` + State *ScheduleState `protobuf:"bytes,4,opt,name=state,proto3" json:"state,omitempty"` +} + +func (x *Schedule) Reset() { + *x = Schedule{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Schedule) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Schedule) ProtoMessage() {} + +func (x *Schedule) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Schedule.ProtoReflect.Descriptor instead. +func (*Schedule) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{13} +} + +func (x *Schedule) GetSpec() *ScheduleSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *Schedule) GetAction() *ScheduleAction { + if x != nil { + return x.Action + } + return nil +} + +func (x *Schedule) GetPolicies() *SchedulePolicies { + if x != nil { + return x.Policies + } + return nil +} + +func (x *Schedule) GetState() *ScheduleState { + if x != nil { + return x.State + } + return nil +} + +// ScheduleListInfo is an abbreviated set of values from Schedule and ScheduleInfo +// that's returned in ListSchedules. +type ScheduleListInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // From spec: + // Some fields are dropped from this copy of spec: timezone_data + Spec *ScheduleSpec `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"` + // From action: + // Action is a oneof field, but we need to encode this in JSON and oneof fields don't work + // well with JSON. If action is start_workflow, this is set: + WorkflowType *v12.WorkflowType `protobuf:"bytes,2,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + // From state: + Notes string `protobuf:"bytes,3,opt,name=notes,proto3" json:"notes,omitempty"` + Paused bool `protobuf:"varint,4,opt,name=paused,proto3" json:"paused,omitempty"` + // From info (maybe fewer entries): + RecentActions []*ScheduleActionResult `protobuf:"bytes,5,rep,name=recent_actions,json=recentActions,proto3" json:"recent_actions,omitempty"` + FutureActionTimes []*timestamppb.Timestamp `protobuf:"bytes,6,rep,name=future_action_times,json=futureActionTimes,proto3" json:"future_action_times,omitempty"` +} + +func (x *ScheduleListInfo) Reset() { + *x = ScheduleListInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ScheduleListInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ScheduleListInfo) ProtoMessage() {} + +func (x *ScheduleListInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ScheduleListInfo.ProtoReflect.Descriptor instead. +func (*ScheduleListInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{14} +} + +func (x *ScheduleListInfo) GetSpec() *ScheduleSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *ScheduleListInfo) GetWorkflowType() *v12.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *ScheduleListInfo) GetNotes() string { + if x != nil { + return x.Notes + } + return "" +} + +func (x *ScheduleListInfo) GetPaused() bool { + if x != nil { + return x.Paused + } + return false +} + +func (x *ScheduleListInfo) GetRecentActions() []*ScheduleActionResult { + if x != nil { + return x.RecentActions + } + return nil +} + +func (x *ScheduleListInfo) GetFutureActionTimes() []*timestamppb.Timestamp { + if x != nil { + return x.FutureActionTimes + } + return nil +} + +// ScheduleListEntry is returned by ListSchedules. +type ScheduleListEntry struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ScheduleId string `protobuf:"bytes,1,opt,name=schedule_id,json=scheduleId,proto3" json:"schedule_id,omitempty"` + Memo *v12.Memo `protobuf:"bytes,2,opt,name=memo,proto3" json:"memo,omitempty"` + SearchAttributes *v12.SearchAttributes `protobuf:"bytes,3,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` + Info *ScheduleListInfo `protobuf:"bytes,4,opt,name=info,proto3" json:"info,omitempty"` +} + +func (x *ScheduleListEntry) Reset() { + *x = ScheduleListEntry{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ScheduleListEntry) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ScheduleListEntry) ProtoMessage() {} + +func (x *ScheduleListEntry) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_schedule_v1_message_proto_msgTypes[15] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ScheduleListEntry.ProtoReflect.Descriptor instead. +func (*ScheduleListEntry) Descriptor() ([]byte, []int) { + return file_temporal_api_schedule_v1_message_proto_rawDescGZIP(), []int{15} +} + +func (x *ScheduleListEntry) GetScheduleId() string { + if x != nil { + return x.ScheduleId + } + return "" +} + +func (x *ScheduleListEntry) GetMemo() *v12.Memo { + if x != nil { + return x.Memo + } + return nil +} + +func (x *ScheduleListEntry) GetSearchAttributes() *v12.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +func (x *ScheduleListEntry) GetInfo() *ScheduleListInfo { + if x != nil { + return x.Info + } + return nil +} + +var File_temporal_api_schedule_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_schedule_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x26, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x73, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x18, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x1a, 0x1e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, + 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, + 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x26, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, + 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xf8, 0x01, 0x0a, 0x0c, 0x43, 0x61, 0x6c, + 0x65, 0x6e, 0x64, 0x61, 0x72, 0x53, 0x70, 0x65, 0x63, 0x12, 0x1a, 0x0a, 0x06, 0x73, 0x65, 0x63, 0x6f, + 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x1a, 0x0a, 0x06, 0x6d, 0x69, 0x6e, 0x75, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x6d, 0x69, 0x6e, 0x75, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x16, 0x0a, + 0x04, 0x68, 0x6f, 0x75, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x6f, 0x75, 0x72, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x24, 0x0a, 0x0c, 0x64, 0x61, 0x79, 0x5f, 0x6f, 0x66, 0x5f, 0x6d, 0x6f, + 0x6e, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x64, 0x61, 0x79, 0x4f, 0x66, 0x4d, + 0x6f, 0x6e, 0x74, 0x68, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x6d, 0x6f, 0x6e, 0x74, 0x68, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6d, 0x6f, 0x6e, 0x74, 0x68, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x16, 0x0a, 0x04, 0x79, 0x65, 0x61, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x79, + 0x65, 0x61, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x22, 0x0a, 0x0b, 0x64, 0x61, 0x79, 0x5f, 0x6f, 0x66, + 0x5f, 0x77, 0x65, 0x65, 0x6b, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, 0x61, 0x79, 0x4f, + 0x66, 0x57, 0x65, 0x65, 0x6b, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, + 0x65, 0x6e, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, + 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0x4f, 0x0a, 0x05, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x18, 0x0a, + 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x14, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x16, 0x0a, 0x04, 0x73, 0x74, + 0x65, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x73, 0x74, 0x65, 0x70, 0x42, 0x02, 0x68, + 0x00, 0x22, 0xe9, 0x03, 0x0a, 0x16, 0x53, 0x74, 0x72, 0x75, 0x63, 0x74, 0x75, 0x72, 0x65, 0x64, 0x43, + 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x53, 0x70, 0x65, 0x63, 0x12, 0x3b, 0x0a, 0x06, 0x73, 0x65, + 0x63, 0x6f, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x06, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x3b, 0x0a, 0x06, 0x6d, 0x69, 0x6e, 0x75, 0x74, 0x65, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x61, 0x6e, + 0x67, 0x65, 0x52, 0x06, 0x6d, 0x69, 0x6e, 0x75, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x37, 0x0a, + 0x04, 0x68, 0x6f, 0x75, 0x72, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x04, 0x68, 0x6f, 0x75, 0x72, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x45, 0x0a, 0x0c, 0x64, 0x61, 0x79, 0x5f, 0x6f, 0x66, 0x5f, 0x6d, 0x6f, 0x6e, + 0x74, 0x68, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x0a, 0x64, 0x61, 0x79, 0x4f, 0x66, 0x4d, 0x6f, 0x6e, + 0x74, 0x68, 0x42, 0x02, 0x68, 0x00, 0x12, 0x39, 0x0a, 0x05, 0x6d, 0x6f, 0x6e, 0x74, 0x68, 0x18, 0x05, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x61, + 0x6e, 0x67, 0x65, 0x52, 0x05, 0x6d, 0x6f, 0x6e, 0x74, 0x68, 0x42, 0x02, 0x68, 0x00, 0x12, 0x37, 0x0a, + 0x04, 0x79, 0x65, 0x61, 0x72, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x04, 0x79, 0x65, 0x61, 0x72, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x43, 0x0a, 0x0b, 0x64, 0x61, 0x79, 0x5f, 0x6f, 0x66, 0x5f, 0x77, 0x65, 0x65, + 0x6b, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x09, 0x64, 0x61, 0x79, 0x4f, 0x66, 0x57, 0x65, 0x65, 0x6b, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x08, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, + 0x22, 0x7e, 0x0a, 0x0c, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x53, 0x70, 0x65, 0x63, 0x12, + 0x39, 0x0a, 0x08, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x65, 0x72, + 0x76, 0x61, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x33, 0x0a, 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, + 0x70, 0x68, 0x61, 0x73, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xfc, 0x05, 0x0a, 0x0c, 0x53, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x65, 0x0a, 0x13, 0x73, 0x74, 0x72, 0x75, + 0x63, 0x74, 0x75, 0x72, 0x65, 0x64, 0x5f, 0x63, 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x18, 0x07, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, + 0x72, 0x75, 0x63, 0x74, 0x75, 0x72, 0x65, 0x64, 0x43, 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x53, + 0x70, 0x65, 0x63, 0x52, 0x12, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x75, 0x72, 0x65, 0x64, 0x43, 0x61, + 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x63, 0x72, 0x6f, + 0x6e, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x63, + 0x72, 0x6f, 0x6e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x42, 0x02, 0x68, 0x00, 0x12, 0x46, 0x0a, 0x08, + 0x63, 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x26, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x53, + 0x70, 0x65, 0x63, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x46, 0x0a, 0x08, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x26, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72, + 0x76, 0x61, 0x6c, 0x53, 0x70, 0x65, 0x63, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x57, 0x0a, 0x10, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x63, + 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x53, 0x70, + 0x65, 0x63, 0x42, 0x04, 0x18, 0x01, 0x68, 0x00, 0x52, 0x0f, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, + 0x65, 0x43, 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x12, 0x74, 0x0a, 0x1b, 0x65, 0x78, 0x63, 0x6c, + 0x75, 0x64, 0x65, 0x5f, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x75, 0x72, 0x65, 0x64, 0x5f, 0x63, 0x61, + 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, + 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x72, 0x75, 0x63, 0x74, 0x75, 0x72, 0x65, 0x64, 0x43, + 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x53, 0x70, 0x65, 0x63, 0x52, 0x19, 0x65, 0x78, 0x63, 0x6c, + 0x75, 0x64, 0x65, 0x53, 0x74, 0x72, 0x75, 0x63, 0x74, 0x75, 0x72, 0x65, 0x64, 0x43, 0x61, 0x6c, 0x65, + 0x6e, 0x64, 0x61, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x39, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x07, + 0x65, 0x6e, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x35, 0x0a, 0x06, 0x6a, 0x69, + 0x74, 0x74, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x6a, 0x69, 0x74, 0x74, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, + 0x0a, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x7a, 0x6f, 0x6e, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0a, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x7a, 0x6f, 0x6e, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, 0x0a, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x7a, 0x6f, 0x6e, 0x65, + 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x74, 0x69, 0x6d, 0x65, + 0x7a, 0x6f, 0x6e, 0x65, 0x44, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x22, 0x9e, 0x02, 0x0a, 0x10, + 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x12, + 0x57, 0x0a, 0x0e, 0x6f, 0x76, 0x65, 0x72, 0x6c, 0x61, 0x70, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4f, 0x76, 0x65, 0x72, 0x6c, 0x61, 0x70, 0x50, 0x6f, 0x6c, 0x69, 0x63, + 0x79, 0x52, 0x0d, 0x6f, 0x76, 0x65, 0x72, 0x6c, 0x61, 0x70, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x44, 0x0a, 0x0e, 0x63, 0x61, 0x74, 0x63, 0x68, 0x75, 0x70, 0x5f, 0x77, 0x69, + 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x63, 0x61, 0x74, 0x63, 0x68, 0x75, 0x70, 0x57, 0x69, 0x6e, 0x64, 0x6f, + 0x77, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x70, 0x61, 0x75, 0x73, 0x65, 0x5f, 0x6f, 0x6e, + 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x70, + 0x61, 0x75, 0x73, 0x65, 0x4f, 0x6e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x3d, 0x0a, 0x19, 0x6b, 0x65, 0x65, 0x70, 0x5f, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x61, 0x6c, + 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x16, 0x6b, 0x65, 0x65, 0x70, 0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x61, 0x6c, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x7b, 0x0a, 0x0e, 0x53, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5f, 0x0a, 0x0e, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x65, 0x77, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x6e, 0x66, 0x6f, 0x48, 0x00, 0x52, 0x0d, 0x73, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x02, 0x68, 0x00, 0x42, 0x08, 0x0a, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x22, 0xff, 0x01, 0x0a, 0x14, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x41, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x43, 0x0a, 0x0d, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, + 0x6c, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, 0x0a, 0x0b, 0x61, 0x63, 0x74, + 0x75, 0x61, 0x6c, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0a, 0x61, 0x63, 0x74, 0x75, 0x61, 0x6c, 0x54, + 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x61, 0x0a, 0x15, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x0b, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x42, 0x02, 0x68, 0x00, 0x22, 0xa3, 0x01, 0x0a, 0x0d, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x18, 0x0a, 0x05, 0x6e, 0x6f, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x6e, 0x6f, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1a, 0x0a, + 0x06, 0x70, 0x61, 0x75, 0x73, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x70, 0x61, + 0x75, 0x73, 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2b, 0x0a, 0x0f, 0x6c, 0x69, 0x6d, 0x69, 0x74, + 0x65, 0x64, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0e, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x64, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x2f, 0x0a, 0x11, 0x72, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x5f, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x72, 0x65, 0x6d, + 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x22, 0x74, 0x0a, 0x19, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x49, 0x6d, 0x6d, 0x65, 0x64, 0x69, + 0x61, 0x74, 0x65, 0x6c, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x57, 0x0a, 0x0e, 0x6f, + 0x76, 0x65, 0x72, 0x6c, 0x61, 0x70, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x4f, 0x76, 0x65, 0x72, 0x6c, 0x61, 0x70, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0d, 0x6f, + 0x76, 0x65, 0x72, 0x6c, 0x61, 0x70, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, + 0xe4, 0x01, 0x0a, 0x0f, 0x42, 0x61, 0x63, 0x6b, 0x66, 0x69, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, + 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x39, 0x0a, + 0x08, 0x65, 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x07, 0x65, 0x6e, 0x64, 0x54, 0x69, 0x6d, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x6f, 0x76, 0x65, 0x72, 0x6c, 0x61, 0x70, 0x5f, + 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4f, 0x76, 0x65, 0x72, 0x6c, 0x61, + 0x70, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0d, 0x6f, 0x76, 0x65, 0x72, 0x6c, 0x61, 0x70, 0x50, + 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8b, 0x02, 0x0a, 0x0d, 0x53, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x63, 0x68, 0x12, 0x68, 0x0a, 0x13, 0x74, 0x72, + 0x69, 0x67, 0x67, 0x65, 0x72, 0x5f, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x6c, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x49, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x6c, + 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x12, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, + 0x49, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x6c, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x58, + 0x0a, 0x10, 0x62, 0x61, 0x63, 0x6b, 0x66, 0x69, 0x6c, 0x6c, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x66, 0x69, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, + 0x0f, 0x62, 0x61, 0x63, 0x6b, 0x66, 0x69, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x70, 0x61, 0x75, 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x70, 0x61, 0x75, 0x73, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x75, + 0x6e, 0x70, 0x61, 0x75, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x75, 0x6e, 0x70, + 0x61, 0x75, 0x73, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xaf, 0x05, 0x0a, 0x0c, 0x53, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x25, 0x0a, 0x0c, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x36, 0x0a, 0x15, + 0x6d, 0x69, 0x73, 0x73, 0x65, 0x64, 0x5f, 0x63, 0x61, 0x74, 0x63, 0x68, 0x75, 0x70, 0x5f, 0x77, 0x69, + 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x13, 0x6d, 0x69, 0x73, 0x73, 0x65, + 0x64, 0x43, 0x61, 0x74, 0x63, 0x68, 0x75, 0x70, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x2b, 0x0a, 0x0f, 0x6f, 0x76, 0x65, 0x72, 0x6c, 0x61, 0x70, 0x5f, 0x73, 0x6b, 0x69, 0x70, + 0x70, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6f, 0x76, 0x65, 0x72, 0x6c, 0x61, + 0x70, 0x53, 0x6b, 0x69, 0x70, 0x70, 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, 0x0a, 0x0e, 0x62, + 0x75, 0x66, 0x66, 0x65, 0x72, 0x5f, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x18, 0x0a, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0d, 0x62, 0x75, 0x66, 0x66, 0x65, 0x72, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x62, 0x75, 0x66, 0x66, 0x65, 0x72, 0x5f, 0x73, 0x69, + 0x7a, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x62, 0x75, 0x66, 0x66, 0x65, 0x72, 0x53, + 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5a, 0x0a, 0x11, 0x72, 0x75, 0x6e, 0x6e, 0x69, 0x6e, + 0x67, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x10, 0x72, 0x75, 0x6e, 0x6e, 0x69, 0x6e, + 0x67, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, + 0x0e, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x52, 0x0d, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x4e, 0x0a, 0x13, 0x66, 0x75, 0x74, 0x75, 0x72, 0x65, 0x5f, 0x61, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x11, 0x66, 0x75, 0x74, 0x75, 0x72, 0x65, + 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, + 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0a, 0x63, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, 0x0a, 0x0b, 0x75, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x16, 0x69, 0x6e, 0x76, 0x61, + 0x6c, 0x69, 0x64, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x42, 0x04, 0x18, 0x01, 0x68, 0x00, 0x52, 0x14, 0x69, 0x6e, + 0x76, 0x61, 0x6c, 0x69, 0x64, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x45, 0x72, 0x72, 0x6f, + 0x72, 0x22, 0x9f, 0x02, 0x0a, 0x08, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x12, 0x3e, 0x0a, + 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x53, 0x70, 0x65, 0x63, + 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x42, 0x02, 0x68, 0x00, 0x12, 0x44, 0x0a, 0x06, 0x61, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x08, 0x70, + 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x50, 0x6f, + 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x52, 0x08, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x27, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x82, 0x03, 0x0a, 0x10, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4c, 0x69, + 0x73, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x3e, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, + 0x6e, 0x6f, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6e, 0x6f, 0x74, 0x65, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1a, 0x0a, 0x06, 0x70, 0x61, 0x75, 0x73, 0x65, 0x64, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x70, 0x61, 0x75, 0x73, 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x59, 0x0a, 0x0e, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x52, 0x0d, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4e, 0x0a, 0x13, 0x66, 0x75, 0x74, 0x75, 0x72, 0x65, 0x5f, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x11, 0x66, 0x75, 0x74, + 0x75, 0x72, 0x65, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x8d, 0x02, 0x0a, 0x11, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4c, 0x69, 0x73, + 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x23, 0x0a, 0x0b, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, 0x04, 0x6d, 0x65, 0x6d, 0x6f, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, + 0x6d, 0x6f, 0x52, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, 0x11, 0x73, + 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x61, + 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x10, 0x73, 0x65, + 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x42, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, + 0x00, 0x42, 0x93, 0x01, 0x0a, 0x1b, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x42, + 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x27, + 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2f, 0x76, 0x31, 0x3b, 0x73, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0xaa, 0x02, 0x1a, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, + 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x56, 0x31, + 0xea, 0x02, 0x1d, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, + 0x69, 0x3a, 0x3a, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_schedule_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_schedule_v1_message_proto_rawDescData = file_temporal_api_schedule_v1_message_proto_rawDesc +) + +func file_temporal_api_schedule_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_schedule_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_schedule_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_schedule_v1_message_proto_rawDescData) + }) + return file_temporal_api_schedule_v1_message_proto_rawDescData +} + +var file_temporal_api_schedule_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 16) +var file_temporal_api_schedule_v1_message_proto_goTypes = []any{ + (*CalendarSpec)(nil), // 0: temporal.api.schedule.v1.CalendarSpec + (*Range)(nil), // 1: temporal.api.schedule.v1.Range + (*StructuredCalendarSpec)(nil), // 2: temporal.api.schedule.v1.StructuredCalendarSpec + (*IntervalSpec)(nil), // 3: temporal.api.schedule.v1.IntervalSpec + (*ScheduleSpec)(nil), // 4: temporal.api.schedule.v1.ScheduleSpec + (*SchedulePolicies)(nil), // 5: temporal.api.schedule.v1.SchedulePolicies + (*ScheduleAction)(nil), // 6: temporal.api.schedule.v1.ScheduleAction + (*ScheduleActionResult)(nil), // 7: temporal.api.schedule.v1.ScheduleActionResult + (*ScheduleState)(nil), // 8: temporal.api.schedule.v1.ScheduleState + (*TriggerImmediatelyRequest)(nil), // 9: temporal.api.schedule.v1.TriggerImmediatelyRequest + (*BackfillRequest)(nil), // 10: temporal.api.schedule.v1.BackfillRequest + (*SchedulePatch)(nil), // 11: temporal.api.schedule.v1.SchedulePatch + (*ScheduleInfo)(nil), // 12: temporal.api.schedule.v1.ScheduleInfo + (*Schedule)(nil), // 13: temporal.api.schedule.v1.Schedule + (*ScheduleListInfo)(nil), // 14: temporal.api.schedule.v1.ScheduleListInfo + (*ScheduleListEntry)(nil), // 15: temporal.api.schedule.v1.ScheduleListEntry + (*durationpb.Duration)(nil), // 16: google.protobuf.Duration + (*timestamppb.Timestamp)(nil), // 17: google.protobuf.Timestamp + (v1.ScheduleOverlapPolicy)(0), // 18: temporal.api.enums.v1.ScheduleOverlapPolicy + (*v11.NewWorkflowExecutionInfo)(nil), // 19: temporal.api.workflow.v1.NewWorkflowExecutionInfo + (*v12.WorkflowExecution)(nil), // 20: temporal.api.common.v1.WorkflowExecution + (*v12.WorkflowType)(nil), // 21: temporal.api.common.v1.WorkflowType + (*v12.Memo)(nil), // 22: temporal.api.common.v1.Memo + (*v12.SearchAttributes)(nil), // 23: temporal.api.common.v1.SearchAttributes +} +var file_temporal_api_schedule_v1_message_proto_depIdxs = []int32{ + 1, // 0: temporal.api.schedule.v1.StructuredCalendarSpec.second:type_name -> temporal.api.schedule.v1.Range + 1, // 1: temporal.api.schedule.v1.StructuredCalendarSpec.minute:type_name -> temporal.api.schedule.v1.Range + 1, // 2: temporal.api.schedule.v1.StructuredCalendarSpec.hour:type_name -> temporal.api.schedule.v1.Range + 1, // 3: temporal.api.schedule.v1.StructuredCalendarSpec.day_of_month:type_name -> temporal.api.schedule.v1.Range + 1, // 4: temporal.api.schedule.v1.StructuredCalendarSpec.month:type_name -> temporal.api.schedule.v1.Range + 1, // 5: temporal.api.schedule.v1.StructuredCalendarSpec.year:type_name -> temporal.api.schedule.v1.Range + 1, // 6: temporal.api.schedule.v1.StructuredCalendarSpec.day_of_week:type_name -> temporal.api.schedule.v1.Range + 16, // 7: temporal.api.schedule.v1.IntervalSpec.interval:type_name -> google.protobuf.Duration + 16, // 8: temporal.api.schedule.v1.IntervalSpec.phase:type_name -> google.protobuf.Duration + 2, // 9: temporal.api.schedule.v1.ScheduleSpec.structured_calendar:type_name -> temporal.api.schedule.v1.StructuredCalendarSpec + 0, // 10: temporal.api.schedule.v1.ScheduleSpec.calendar:type_name -> temporal.api.schedule.v1.CalendarSpec + 3, // 11: temporal.api.schedule.v1.ScheduleSpec.interval:type_name -> temporal.api.schedule.v1.IntervalSpec + 0, // 12: temporal.api.schedule.v1.ScheduleSpec.exclude_calendar:type_name -> temporal.api.schedule.v1.CalendarSpec + 2, // 13: temporal.api.schedule.v1.ScheduleSpec.exclude_structured_calendar:type_name -> temporal.api.schedule.v1.StructuredCalendarSpec + 17, // 14: temporal.api.schedule.v1.ScheduleSpec.start_time:type_name -> google.protobuf.Timestamp + 17, // 15: temporal.api.schedule.v1.ScheduleSpec.end_time:type_name -> google.protobuf.Timestamp + 16, // 16: temporal.api.schedule.v1.ScheduleSpec.jitter:type_name -> google.protobuf.Duration + 18, // 17: temporal.api.schedule.v1.SchedulePolicies.overlap_policy:type_name -> temporal.api.enums.v1.ScheduleOverlapPolicy + 16, // 18: temporal.api.schedule.v1.SchedulePolicies.catchup_window:type_name -> google.protobuf.Duration + 19, // 19: temporal.api.schedule.v1.ScheduleAction.start_workflow:type_name -> temporal.api.workflow.v1.NewWorkflowExecutionInfo + 17, // 20: temporal.api.schedule.v1.ScheduleActionResult.schedule_time:type_name -> google.protobuf.Timestamp + 17, // 21: temporal.api.schedule.v1.ScheduleActionResult.actual_time:type_name -> google.protobuf.Timestamp + 20, // 22: temporal.api.schedule.v1.ScheduleActionResult.start_workflow_result:type_name -> temporal.api.common.v1.WorkflowExecution + 18, // 23: temporal.api.schedule.v1.TriggerImmediatelyRequest.overlap_policy:type_name -> temporal.api.enums.v1.ScheduleOverlapPolicy + 17, // 24: temporal.api.schedule.v1.BackfillRequest.start_time:type_name -> google.protobuf.Timestamp + 17, // 25: temporal.api.schedule.v1.BackfillRequest.end_time:type_name -> google.protobuf.Timestamp + 18, // 26: temporal.api.schedule.v1.BackfillRequest.overlap_policy:type_name -> temporal.api.enums.v1.ScheduleOverlapPolicy + 9, // 27: temporal.api.schedule.v1.SchedulePatch.trigger_immediately:type_name -> temporal.api.schedule.v1.TriggerImmediatelyRequest + 10, // 28: temporal.api.schedule.v1.SchedulePatch.backfill_request:type_name -> temporal.api.schedule.v1.BackfillRequest + 20, // 29: temporal.api.schedule.v1.ScheduleInfo.running_workflows:type_name -> temporal.api.common.v1.WorkflowExecution + 7, // 30: temporal.api.schedule.v1.ScheduleInfo.recent_actions:type_name -> temporal.api.schedule.v1.ScheduleActionResult + 17, // 31: temporal.api.schedule.v1.ScheduleInfo.future_action_times:type_name -> google.protobuf.Timestamp + 17, // 32: temporal.api.schedule.v1.ScheduleInfo.create_time:type_name -> google.protobuf.Timestamp + 17, // 33: temporal.api.schedule.v1.ScheduleInfo.update_time:type_name -> google.protobuf.Timestamp + 4, // 34: temporal.api.schedule.v1.Schedule.spec:type_name -> temporal.api.schedule.v1.ScheduleSpec + 6, // 35: temporal.api.schedule.v1.Schedule.action:type_name -> temporal.api.schedule.v1.ScheduleAction + 5, // 36: temporal.api.schedule.v1.Schedule.policies:type_name -> temporal.api.schedule.v1.SchedulePolicies + 8, // 37: temporal.api.schedule.v1.Schedule.state:type_name -> temporal.api.schedule.v1.ScheduleState + 4, // 38: temporal.api.schedule.v1.ScheduleListInfo.spec:type_name -> temporal.api.schedule.v1.ScheduleSpec + 21, // 39: temporal.api.schedule.v1.ScheduleListInfo.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 7, // 40: temporal.api.schedule.v1.ScheduleListInfo.recent_actions:type_name -> temporal.api.schedule.v1.ScheduleActionResult + 17, // 41: temporal.api.schedule.v1.ScheduleListInfo.future_action_times:type_name -> google.protobuf.Timestamp + 22, // 42: temporal.api.schedule.v1.ScheduleListEntry.memo:type_name -> temporal.api.common.v1.Memo + 23, // 43: temporal.api.schedule.v1.ScheduleListEntry.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 14, // 44: temporal.api.schedule.v1.ScheduleListEntry.info:type_name -> temporal.api.schedule.v1.ScheduleListInfo + 45, // [45:45] is the sub-list for method output_type + 45, // [45:45] is the sub-list for method input_type + 45, // [45:45] is the sub-list for extension type_name + 45, // [45:45] is the sub-list for extension extendee + 0, // [0:45] is the sub-list for field type_name +} + +func init() { file_temporal_api_schedule_v1_message_proto_init() } +func file_temporal_api_schedule_v1_message_proto_init() { + if File_temporal_api_schedule_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_schedule_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*CalendarSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*Range); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*StructuredCalendarSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*IntervalSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*ScheduleSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*SchedulePolicies); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*ScheduleAction); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*ScheduleActionResult); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*ScheduleState); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*TriggerImmediatelyRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*BackfillRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[11].Exporter = func(v any, i int) any { + switch v := v.(*SchedulePatch); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[12].Exporter = func(v any, i int) any { + switch v := v.(*ScheduleInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[13].Exporter = func(v any, i int) any { + switch v := v.(*Schedule); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[14].Exporter = func(v any, i int) any { + switch v := v.(*ScheduleListInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[15].Exporter = func(v any, i int) any { + switch v := v.(*ScheduleListEntry); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_temporal_api_schedule_v1_message_proto_msgTypes[6].OneofWrappers = []any{ + (*ScheduleAction_StartWorkflow)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_schedule_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 16, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_schedule_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_schedule_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_schedule_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_schedule_v1_message_proto = out.File + file_temporal_api_schedule_v1_message_proto_rawDesc = nil + file_temporal_api_schedule_v1_message_proto_goTypes = nil + file_temporal_api_schedule_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/sdk/v1/enhanced_stack_trace.go-helpers.pb.go b/vendor/go.temporal.io/api/sdk/v1/enhanced_stack_trace.go-helpers.pb.go new file mode 100644 index 00000000000..693dba3333f --- /dev/null +++ b/vendor/go.temporal.io/api/sdk/v1/enhanced_stack_trace.go-helpers.pb.go @@ -0,0 +1,213 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package sdk + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type EnhancedStackTrace to the protobuf v3 wire format +func (val *EnhancedStackTrace) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type EnhancedStackTrace from the protobuf v3 wire format +func (val *EnhancedStackTrace) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *EnhancedStackTrace) Size() int { + return proto.Size(val) +} + +// Equal returns whether two EnhancedStackTrace values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *EnhancedStackTrace) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *EnhancedStackTrace + switch t := that.(type) { + case *EnhancedStackTrace: + that1 = t + case EnhancedStackTrace: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StackTraceSDKInfo to the protobuf v3 wire format +func (val *StackTraceSDKInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StackTraceSDKInfo from the protobuf v3 wire format +func (val *StackTraceSDKInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StackTraceSDKInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StackTraceSDKInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StackTraceSDKInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StackTraceSDKInfo + switch t := that.(type) { + case *StackTraceSDKInfo: + that1 = t + case StackTraceSDKInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StackTraceFileSlice to the protobuf v3 wire format +func (val *StackTraceFileSlice) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StackTraceFileSlice from the protobuf v3 wire format +func (val *StackTraceFileSlice) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StackTraceFileSlice) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StackTraceFileSlice values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StackTraceFileSlice) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StackTraceFileSlice + switch t := that.(type) { + case *StackTraceFileSlice: + that1 = t + case StackTraceFileSlice: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StackTraceFileLocation to the protobuf v3 wire format +func (val *StackTraceFileLocation) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StackTraceFileLocation from the protobuf v3 wire format +func (val *StackTraceFileLocation) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StackTraceFileLocation) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StackTraceFileLocation values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StackTraceFileLocation) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StackTraceFileLocation + switch t := that.(type) { + case *StackTraceFileLocation: + that1 = t + case StackTraceFileLocation: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StackTrace to the protobuf v3 wire format +func (val *StackTrace) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StackTrace from the protobuf v3 wire format +func (val *StackTrace) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StackTrace) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StackTrace values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StackTrace) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StackTrace + switch t := that.(type) { + case *StackTrace: + that1 = t + case StackTrace: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/sdk/v1/enhanced_stack_trace.pb.go b/vendor/go.temporal.io/api/sdk/v1/enhanced_stack_trace.pb.go new file mode 100644 index 00000000000..08d279fa452 --- /dev/null +++ b/vendor/go.temporal.io/api/sdk/v1/enhanced_stack_trace.pb.go @@ -0,0 +1,559 @@ +// The MIT License +// +// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/sdk/v1/enhanced_stack_trace.proto + +package sdk + +import ( + reflect "reflect" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// Internal structure used to create worker stack traces with references to code. +type EnhancedStackTrace struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Information pertaining to the SDK that the trace has been captured from. + Sdk *StackTraceSDKInfo `protobuf:"bytes,1,opt,name=sdk,proto3" json:"sdk,omitempty"` + // Mapping of file path to file contents. + Sources map[string]*StackTraceFileSlice `protobuf:"bytes,2,rep,name=sources,proto3" json:"sources,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // Collection of stacks captured. + Stacks []*StackTrace `protobuf:"bytes,3,rep,name=stacks,proto3" json:"stacks,omitempty"` +} + +func (x *EnhancedStackTrace) Reset() { + *x = EnhancedStackTrace{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *EnhancedStackTrace) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EnhancedStackTrace) ProtoMessage() {} + +func (x *EnhancedStackTrace) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EnhancedStackTrace.ProtoReflect.Descriptor instead. +func (*EnhancedStackTrace) Descriptor() ([]byte, []int) { + return file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDescGZIP(), []int{0} +} + +func (x *EnhancedStackTrace) GetSdk() *StackTraceSDKInfo { + if x != nil { + return x.Sdk + } + return nil +} + +func (x *EnhancedStackTrace) GetSources() map[string]*StackTraceFileSlice { + if x != nil { + return x.Sources + } + return nil +} + +func (x *EnhancedStackTrace) GetStacks() []*StackTrace { + if x != nil { + return x.Stacks + } + return nil +} + +// Information pertaining to the SDK that the trace has been captured from. +// (-- api-linter: core::0123::resource-annotation=disabled +// +// aip.dev/not-precedent: Naming SDK version is optional. --) +type StackTraceSDKInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Name of the SDK + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // Version string of the SDK + Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"` +} + +func (x *StackTraceSDKInfo) Reset() { + *x = StackTraceSDKInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StackTraceSDKInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StackTraceSDKInfo) ProtoMessage() {} + +func (x *StackTraceSDKInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StackTraceSDKInfo.ProtoReflect.Descriptor instead. +func (*StackTraceSDKInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDescGZIP(), []int{1} +} + +func (x *StackTraceSDKInfo) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *StackTraceSDKInfo) GetVersion() string { + if x != nil { + return x.Version + } + return "" +} + +// "Slice" of a file starting at line_offset -- a line offset and code fragment corresponding to the worker's stack. +type StackTraceFileSlice struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Only used (possibly) to trim the file without breaking syntax highlighting. This is not optional, unlike + // the `line` property of a `StackTraceFileLocation`. + // (-- api-linter: core::0141::forbidden-types=disabled + // + // aip.dev/not-precedent: These really shouldn't have negative values. --) + LineOffset uint32 `protobuf:"varint,1,opt,name=line_offset,json=lineOffset,proto3" json:"line_offset,omitempty"` + // Slice of a file with the respective OS-specific line terminator. + Content string `protobuf:"bytes,2,opt,name=content,proto3" json:"content,omitempty"` +} + +func (x *StackTraceFileSlice) Reset() { + *x = StackTraceFileSlice{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StackTraceFileSlice) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StackTraceFileSlice) ProtoMessage() {} + +func (x *StackTraceFileSlice) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StackTraceFileSlice.ProtoReflect.Descriptor instead. +func (*StackTraceFileSlice) Descriptor() ([]byte, []int) { + return file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDescGZIP(), []int{2} +} + +func (x *StackTraceFileSlice) GetLineOffset() uint32 { + if x != nil { + return x.LineOffset + } + return 0 +} + +func (x *StackTraceFileSlice) GetContent() string { + if x != nil { + return x.Content + } + return "" +} + +// More specific location details of a file: its path, precise line and column numbers if applicable, and function name if available. +// In essence, a pointer to a location in a file +type StackTraceFileLocation struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Path to source file (absolute or relative). + // If the paths are relative, ensure that they are all relative to the same root. + FilePath string `protobuf:"bytes,1,opt,name=file_path,json=filePath,proto3" json:"file_path,omitempty"` + // Optional; If possible, SDK should send this -- this is required for displaying the code location. + // If not provided, set to -1. + Line int32 `protobuf:"varint,2,opt,name=line,proto3" json:"line,omitempty"` + // Optional; if possible, SDK should send this. + // If not provided, set to -1. + Column int32 `protobuf:"varint,3,opt,name=column,proto3" json:"column,omitempty"` + // Function name this line belongs to, if applicable. + // Used for falling back to stack trace view. + FunctionName string `protobuf:"bytes,4,opt,name=function_name,json=functionName,proto3" json:"function_name,omitempty"` + // Flag to communicate whether a location should be hidden by default in the stack view. + InternalCode bool `protobuf:"varint,5,opt,name=internal_code,json=internalCode,proto3" json:"internal_code,omitempty"` +} + +func (x *StackTraceFileLocation) Reset() { + *x = StackTraceFileLocation{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StackTraceFileLocation) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StackTraceFileLocation) ProtoMessage() {} + +func (x *StackTraceFileLocation) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StackTraceFileLocation.ProtoReflect.Descriptor instead. +func (*StackTraceFileLocation) Descriptor() ([]byte, []int) { + return file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDescGZIP(), []int{3} +} + +func (x *StackTraceFileLocation) GetFilePath() string { + if x != nil { + return x.FilePath + } + return "" +} + +func (x *StackTraceFileLocation) GetLine() int32 { + if x != nil { + return x.Line + } + return 0 +} + +func (x *StackTraceFileLocation) GetColumn() int32 { + if x != nil { + return x.Column + } + return 0 +} + +func (x *StackTraceFileLocation) GetFunctionName() string { + if x != nil { + return x.FunctionName + } + return "" +} + +func (x *StackTraceFileLocation) GetInternalCode() bool { + if x != nil { + return x.InternalCode + } + return false +} + +// Collection of FileLocation messages from a single stack. +type StackTrace struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Collection of `FileLocation`s, each for a stack frame that comprise a stack trace. + Locations []*StackTraceFileLocation `protobuf:"bytes,1,rep,name=locations,proto3" json:"locations,omitempty"` +} + +func (x *StackTrace) Reset() { + *x = StackTrace{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StackTrace) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StackTrace) ProtoMessage() {} + +func (x *StackTrace) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StackTrace.ProtoReflect.Descriptor instead. +func (*StackTrace) Descriptor() ([]byte, []int) { + return file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDescGZIP(), []int{4} +} + +func (x *StackTrace) GetLocations() []*StackTraceFileLocation { + if x != nil { + return x.Locations + } + return nil +} + +var File_temporal_api_sdk_v1_enhanced_stack_trace_proto protoreflect.FileDescriptor + +var file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDesc = []byte{ + 0x0a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x73, 0x64, 0x6b, + 0x2f, 0x76, 0x31, 0x2f, 0x65, 0x6e, 0x68, 0x61, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x73, 0x74, 0x61, 0x63, + 0x6b, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x13, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, + 0x22, 0xd1, 0x02, 0x0a, 0x12, 0x45, 0x6e, 0x68, 0x61, 0x6e, 0x63, 0x65, 0x64, 0x53, 0x74, 0x61, 0x63, + 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x12, 0x3c, 0x0a, 0x03, 0x73, 0x64, 0x6b, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x63, 0x6b, 0x54, 0x72, 0x61, 0x63, + 0x65, 0x53, 0x44, 0x4b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x03, 0x73, 0x64, 0x6b, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x52, 0x0a, 0x07, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x34, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, + 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x68, 0x61, 0x6e, 0x63, 0x65, 0x64, 0x53, 0x74, 0x61, + 0x63, 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x07, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x3b, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x63, 0x6b, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, + 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x63, 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x52, 0x06, 0x73, + 0x74, 0x61, 0x63, 0x6b, 0x73, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x6c, 0x0a, 0x0c, 0x53, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x42, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x74, 0x61, 0x63, 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x6c, 0x69, + 0x63, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0x49, 0x0a, 0x11, 0x53, 0x74, 0x61, 0x63, 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x53, 0x44, 0x4b, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x16, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x58, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x63, 0x6b, 0x54, + 0x72, 0x61, 0x63, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x6c, 0x69, 0x63, 0x65, 0x12, 0x23, 0x0a, 0x0b, + 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, + 0x52, 0x0a, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x1c, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0xbf, 0x01, 0x0a, 0x16, + 0x53, 0x74, 0x61, 0x63, 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x4c, 0x6f, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x74, + 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x16, 0x0a, 0x04, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x04, 0x6c, 0x69, 0x6e, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1a, 0x0a, 0x06, 0x63, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x75, 0x6e, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, 0x0a, 0x0d, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x0c, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6f, 0x64, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x5b, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x63, 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x12, + 0x4d, 0x0a, 0x09, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x2b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x63, 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, + 0x46, 0x69, 0x6c, 0x65, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x6c, 0x6f, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x42, 0x85, 0x01, 0x0a, 0x16, 0x69, 0x6f, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, + 0x2e, 0x76, 0x31, 0x42, 0x17, 0x45, 0x6e, 0x68, 0x61, 0x6e, 0x63, 0x65, 0x64, 0x53, 0x74, 0x61, 0x63, + 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x1d, 0x67, 0x6f, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, + 0x73, 0x64, 0x6b, 0x2f, 0x76, 0x31, 0x3b, 0x73, 0x64, 0x6b, 0xaa, 0x02, 0x15, 0x54, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x53, 0x64, 0x6b, 0x2e, 0x56, 0x31, + 0xea, 0x02, 0x18, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, + 0x69, 0x3a, 0x3a, 0x53, 0x64, 0x6b, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x33, +} + +var ( + file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDescOnce sync.Once + file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDescData = file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDesc +) + +func file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDescGZIP() []byte { + file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDescOnce.Do(func() { + file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDescData) + }) + return file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDescData +} + +var file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes = make([]protoimpl.MessageInfo, 6) +var file_temporal_api_sdk_v1_enhanced_stack_trace_proto_goTypes = []any{ + (*EnhancedStackTrace)(nil), // 0: temporal.api.sdk.v1.EnhancedStackTrace + (*StackTraceSDKInfo)(nil), // 1: temporal.api.sdk.v1.StackTraceSDKInfo + (*StackTraceFileSlice)(nil), // 2: temporal.api.sdk.v1.StackTraceFileSlice + (*StackTraceFileLocation)(nil), // 3: temporal.api.sdk.v1.StackTraceFileLocation + (*StackTrace)(nil), // 4: temporal.api.sdk.v1.StackTrace + nil, // 5: temporal.api.sdk.v1.EnhancedStackTrace.SourcesEntry +} +var file_temporal_api_sdk_v1_enhanced_stack_trace_proto_depIdxs = []int32{ + 1, // 0: temporal.api.sdk.v1.EnhancedStackTrace.sdk:type_name -> temporal.api.sdk.v1.StackTraceSDKInfo + 5, // 1: temporal.api.sdk.v1.EnhancedStackTrace.sources:type_name -> temporal.api.sdk.v1.EnhancedStackTrace.SourcesEntry + 4, // 2: temporal.api.sdk.v1.EnhancedStackTrace.stacks:type_name -> temporal.api.sdk.v1.StackTrace + 3, // 3: temporal.api.sdk.v1.StackTrace.locations:type_name -> temporal.api.sdk.v1.StackTraceFileLocation + 2, // 4: temporal.api.sdk.v1.EnhancedStackTrace.SourcesEntry.value:type_name -> temporal.api.sdk.v1.StackTraceFileSlice + 5, // [5:5] is the sub-list for method output_type + 5, // [5:5] is the sub-list for method input_type + 5, // [5:5] is the sub-list for extension type_name + 5, // [5:5] is the sub-list for extension extendee + 0, // [0:5] is the sub-list for field type_name +} + +func init() { file_temporal_api_sdk_v1_enhanced_stack_trace_proto_init() } +func file_temporal_api_sdk_v1_enhanced_stack_trace_proto_init() { + if File_temporal_api_sdk_v1_enhanced_stack_trace_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*EnhancedStackTrace); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*StackTraceSDKInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*StackTraceFileSlice); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*StackTraceFileLocation); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*StackTrace); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDesc, + NumEnums: 0, + NumMessages: 6, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_sdk_v1_enhanced_stack_trace_proto_goTypes, + DependencyIndexes: file_temporal_api_sdk_v1_enhanced_stack_trace_proto_depIdxs, + MessageInfos: file_temporal_api_sdk_v1_enhanced_stack_trace_proto_msgTypes, + }.Build() + File_temporal_api_sdk_v1_enhanced_stack_trace_proto = out.File + file_temporal_api_sdk_v1_enhanced_stack_trace_proto_rawDesc = nil + file_temporal_api_sdk_v1_enhanced_stack_trace_proto_goTypes = nil + file_temporal_api_sdk_v1_enhanced_stack_trace_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/sdk/v1/task_complete_metadata.go-helpers.pb.go b/vendor/go.temporal.io/api/sdk/v1/task_complete_metadata.go-helpers.pb.go new file mode 100644 index 00000000000..97f8792ed29 --- /dev/null +++ b/vendor/go.temporal.io/api/sdk/v1/task_complete_metadata.go-helpers.pb.go @@ -0,0 +1,65 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package sdk + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type WorkflowTaskCompletedMetadata to the protobuf v3 wire format +func (val *WorkflowTaskCompletedMetadata) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowTaskCompletedMetadata from the protobuf v3 wire format +func (val *WorkflowTaskCompletedMetadata) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowTaskCompletedMetadata) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowTaskCompletedMetadata values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowTaskCompletedMetadata) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowTaskCompletedMetadata + switch t := that.(type) { + case *WorkflowTaskCompletedMetadata: + that1 = t + case WorkflowTaskCompletedMetadata: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/sdk/v1/task_complete_metadata.pb.go b/vendor/go.temporal.io/api/sdk/v1/task_complete_metadata.pb.go new file mode 100644 index 00000000000..6807113e425 --- /dev/null +++ b/vendor/go.temporal.io/api/sdk/v1/task_complete_metadata.pb.go @@ -0,0 +1,245 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/sdk/v1/task_complete_metadata.proto + +package sdk + +import ( + reflect "reflect" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type WorkflowTaskCompletedMetadata struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Internal flags used by the core SDK. SDKs using flags must comply with the following behavior: + // + // During replay: + // - If a flag is not recognized (value is too high or not defined), it must fail the workflow + // task. + // - If a flag is recognized, it is stored in a set of used flags for the run. Code checks for + // that flag during and after this WFT are allowed to assume that the flag is present. + // - If a code check for a flag does not find the flag in the set of used flags, it must take + // the branch corresponding to the absence of that flag. + // + // During non-replay execution of new WFTs: + // - The SDK is free to use all flags it knows about. It must record any newly-used (IE: not + // previously recorded) flags when completing the WFT. + // + // SDKs which are too old to even know about this field at all are considered to produce + // undefined behavior if they replay workflows which used this mechanism. + // + // (-- api-linter: core::0141::forbidden-types=disabled + // + // aip.dev/not-precedent: These really shouldn't have negative values. --) + CoreUsedFlags []uint32 `protobuf:"varint,1,rep,packed,name=core_used_flags,json=coreUsedFlags,proto3" json:"core_used_flags,omitempty"` + // Flags used by the SDK lang. No attempt is made to distinguish between different SDK languages + // here as processing a workflow with a different language than the one which authored it is + // already undefined behavior. See `core_used_patches` for more. + // + // (-- api-linter: core::0141::forbidden-types=disabled + // + // aip.dev/not-precedent: These really shouldn't have negative values. --) + LangUsedFlags []uint32 `protobuf:"varint,2,rep,packed,name=lang_used_flags,json=langUsedFlags,proto3" json:"lang_used_flags,omitempty"` + // Name of the SDK that processed the task. This is usually something like "temporal-go" and is + // usually the same as client-name gRPC header. This should only be set if its value changed + // since the last time recorded on the workflow (or be set on the first task). + // + // (-- api-linter: core::0122::name-suffix=disabled + // + // aip.dev/not-precedent: We're ok with a name suffix here. --) + SdkName string `protobuf:"bytes,3,opt,name=sdk_name,json=sdkName,proto3" json:"sdk_name,omitempty"` + // Version of the SDK that processed the task. This is usually something like "1.20.0" and is + // usually the same as client-version gRPC header. This should only be set if its value changed + // since the last time recorded on the workflow (or be set on the first task). + SdkVersion string `protobuf:"bytes,4,opt,name=sdk_version,json=sdkVersion,proto3" json:"sdk_version,omitempty"` +} + +func (x *WorkflowTaskCompletedMetadata) Reset() { + *x = WorkflowTaskCompletedMetadata{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_sdk_v1_task_complete_metadata_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowTaskCompletedMetadata) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowTaskCompletedMetadata) ProtoMessage() {} + +func (x *WorkflowTaskCompletedMetadata) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_sdk_v1_task_complete_metadata_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowTaskCompletedMetadata.ProtoReflect.Descriptor instead. +func (*WorkflowTaskCompletedMetadata) Descriptor() ([]byte, []int) { + return file_temporal_api_sdk_v1_task_complete_metadata_proto_rawDescGZIP(), []int{0} +} + +func (x *WorkflowTaskCompletedMetadata) GetCoreUsedFlags() []uint32 { + if x != nil { + return x.CoreUsedFlags + } + return nil +} + +func (x *WorkflowTaskCompletedMetadata) GetLangUsedFlags() []uint32 { + if x != nil { + return x.LangUsedFlags + } + return nil +} + +func (x *WorkflowTaskCompletedMetadata) GetSdkName() string { + if x != nil { + return x.SdkName + } + return "" +} + +func (x *WorkflowTaskCompletedMetadata) GetSdkVersion() string { + if x != nil { + return x.SdkVersion + } + return "" +} + +var File_temporal_api_sdk_v1_task_complete_metadata_proto protoreflect.FileDescriptor + +var file_temporal_api_sdk_v1_task_complete_metadata_proto_rawDesc = []byte{ + 0x0a, 0x30, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x73, 0x64, + 0x6b, 0x2f, 0x76, 0x31, 0x2f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x12, 0x13, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, + 0x6b, 0x2e, 0x76, 0x31, 0x22, 0xbb, 0x01, 0x0a, 0x1d, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x2a, 0x0a, 0x0f, 0x63, 0x6f, 0x72, 0x65, 0x5f, 0x75, 0x73, 0x65, + 0x64, 0x5f, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x0d, 0x63, 0x6f, + 0x72, 0x65, 0x55, 0x73, 0x65, 0x64, 0x46, 0x6c, 0x61, 0x67, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x2a, 0x0a, 0x0f, 0x6c, 0x61, 0x6e, 0x67, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x5f, 0x66, 0x6c, 0x61, 0x67, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x0d, 0x6c, 0x61, 0x6e, 0x67, 0x55, 0x73, 0x65, 0x64, + 0x46, 0x6c, 0x61, 0x67, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1d, 0x0a, 0x08, 0x73, 0x64, 0x6b, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x64, 0x6b, 0x4e, + 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x73, 0x64, 0x6b, 0x5f, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x64, 0x6b, + 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x42, 0x87, 0x01, 0x0a, 0x16, 0x69, + 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, + 0x6b, 0x2e, 0x76, 0x31, 0x42, 0x19, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, + 0x74, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, + 0x5a, 0x1d, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x73, 0x64, 0x6b, 0x2f, 0x76, 0x31, 0x3b, 0x73, 0x64, 0x6b, 0xaa, 0x02, + 0x15, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x53, + 0x64, 0x6b, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x18, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, + 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x53, 0x64, 0x6b, 0x3a, 0x3a, 0x56, 0x31, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_sdk_v1_task_complete_metadata_proto_rawDescOnce sync.Once + file_temporal_api_sdk_v1_task_complete_metadata_proto_rawDescData = file_temporal_api_sdk_v1_task_complete_metadata_proto_rawDesc +) + +func file_temporal_api_sdk_v1_task_complete_metadata_proto_rawDescGZIP() []byte { + file_temporal_api_sdk_v1_task_complete_metadata_proto_rawDescOnce.Do(func() { + file_temporal_api_sdk_v1_task_complete_metadata_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_sdk_v1_task_complete_metadata_proto_rawDescData) + }) + return file_temporal_api_sdk_v1_task_complete_metadata_proto_rawDescData +} + +var file_temporal_api_sdk_v1_task_complete_metadata_proto_msgTypes = make([]protoimpl.MessageInfo, 1) +var file_temporal_api_sdk_v1_task_complete_metadata_proto_goTypes = []any{ + (*WorkflowTaskCompletedMetadata)(nil), // 0: temporal.api.sdk.v1.WorkflowTaskCompletedMetadata +} +var file_temporal_api_sdk_v1_task_complete_metadata_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_sdk_v1_task_complete_metadata_proto_init() } +func file_temporal_api_sdk_v1_task_complete_metadata_proto_init() { + if File_temporal_api_sdk_v1_task_complete_metadata_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_sdk_v1_task_complete_metadata_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowTaskCompletedMetadata); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_sdk_v1_task_complete_metadata_proto_rawDesc, + NumEnums: 0, + NumMessages: 1, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_sdk_v1_task_complete_metadata_proto_goTypes, + DependencyIndexes: file_temporal_api_sdk_v1_task_complete_metadata_proto_depIdxs, + MessageInfos: file_temporal_api_sdk_v1_task_complete_metadata_proto_msgTypes, + }.Build() + File_temporal_api_sdk_v1_task_complete_metadata_proto = out.File + file_temporal_api_sdk_v1_task_complete_metadata_proto_rawDesc = nil + file_temporal_api_sdk_v1_task_complete_metadata_proto_goTypes = nil + file_temporal_api_sdk_v1_task_complete_metadata_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/sdk/v1/user_metadata.go-helpers.pb.go b/vendor/go.temporal.io/api/sdk/v1/user_metadata.go-helpers.pb.go new file mode 100644 index 00000000000..f7c02dfebcd --- /dev/null +++ b/vendor/go.temporal.io/api/sdk/v1/user_metadata.go-helpers.pb.go @@ -0,0 +1,65 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package sdk + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type UserMetadata to the protobuf v3 wire format +func (val *UserMetadata) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UserMetadata from the protobuf v3 wire format +func (val *UserMetadata) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UserMetadata) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UserMetadata values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UserMetadata) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UserMetadata + switch t := that.(type) { + case *UserMetadata: + that1 = t + case UserMetadata: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/sdk/v1/user_metadata.pb.go b/vendor/go.temporal.io/api/sdk/v1/user_metadata.pb.go new file mode 100644 index 00000000000..73f9f105ca2 --- /dev/null +++ b/vendor/go.temporal.io/api/sdk/v1/user_metadata.pb.go @@ -0,0 +1,203 @@ +// The MIT License +// +// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/sdk/v1/user_metadata.proto + +package sdk + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/common/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// Information a user can set, often for use by user interfaces. +type UserMetadata struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Short-form text that provides a summary. This payload should be a "json/plain"-encoded payload + // that is a single JSON string for use in user interfaces. User interface formatting may not + // apply to this text when used in "title" situations. The payload data section is limited to 400 + // bytes by default. + Summary *v1.Payload `protobuf:"bytes,1,opt,name=summary,proto3" json:"summary,omitempty"` + // Long-form text that provides details. This payload should be a "json/plain"-encoded payload + // that is a single JSON string for use in user interfaces. User interface formatting may apply to + // this text in common use. The payload data section is limited to 20000 bytes by default. + Details *v1.Payload `protobuf:"bytes,2,opt,name=details,proto3" json:"details,omitempty"` +} + +func (x *UserMetadata) Reset() { + *x = UserMetadata{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_sdk_v1_user_metadata_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UserMetadata) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UserMetadata) ProtoMessage() {} + +func (x *UserMetadata) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_sdk_v1_user_metadata_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UserMetadata.ProtoReflect.Descriptor instead. +func (*UserMetadata) Descriptor() ([]byte, []int) { + return file_temporal_api_sdk_v1_user_metadata_proto_rawDescGZIP(), []int{0} +} + +func (x *UserMetadata) GetSummary() *v1.Payload { + if x != nil { + return x.Summary + } + return nil +} + +func (x *UserMetadata) GetDetails() *v1.Payload { + if x != nil { + return x.Details + } + return nil +} + +var File_temporal_api_sdk_v1_user_metadata_proto protoreflect.FileDescriptor + +var file_temporal_api_sdk_v1_user_metadata_proto_rawDesc = []byte{ + 0x0a, 0x27, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x73, 0x64, + 0x6b, 0x2f, 0x76, 0x31, 0x2f, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x13, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x1a, 0x24, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x22, 0x8c, 0x01, 0x0a, 0x0c, 0x55, 0x73, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x12, 0x3d, 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x3d, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, + 0x64, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, 0x42, 0x7f, 0x0a, + 0x16, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x42, 0x11, 0x55, 0x73, 0x65, 0x72, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x1d, 0x67, 0x6f, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, + 0x73, 0x64, 0x6b, 0x2f, 0x76, 0x31, 0x3b, 0x73, 0x64, 0x6b, 0xaa, 0x02, 0x15, 0x54, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x53, 0x64, 0x6b, 0x2e, 0x56, + 0x31, 0xea, 0x02, 0x18, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, + 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x53, 0x64, 0x6b, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_sdk_v1_user_metadata_proto_rawDescOnce sync.Once + file_temporal_api_sdk_v1_user_metadata_proto_rawDescData = file_temporal_api_sdk_v1_user_metadata_proto_rawDesc +) + +func file_temporal_api_sdk_v1_user_metadata_proto_rawDescGZIP() []byte { + file_temporal_api_sdk_v1_user_metadata_proto_rawDescOnce.Do(func() { + file_temporal_api_sdk_v1_user_metadata_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_sdk_v1_user_metadata_proto_rawDescData) + }) + return file_temporal_api_sdk_v1_user_metadata_proto_rawDescData +} + +var file_temporal_api_sdk_v1_user_metadata_proto_msgTypes = make([]protoimpl.MessageInfo, 1) +var file_temporal_api_sdk_v1_user_metadata_proto_goTypes = []any{ + (*UserMetadata)(nil), // 0: temporal.api.sdk.v1.UserMetadata + (*v1.Payload)(nil), // 1: temporal.api.common.v1.Payload +} +var file_temporal_api_sdk_v1_user_metadata_proto_depIdxs = []int32{ + 1, // 0: temporal.api.sdk.v1.UserMetadata.summary:type_name -> temporal.api.common.v1.Payload + 1, // 1: temporal.api.sdk.v1.UserMetadata.details:type_name -> temporal.api.common.v1.Payload + 2, // [2:2] is the sub-list for method output_type + 2, // [2:2] is the sub-list for method input_type + 2, // [2:2] is the sub-list for extension type_name + 2, // [2:2] is the sub-list for extension extendee + 0, // [0:2] is the sub-list for field type_name +} + +func init() { file_temporal_api_sdk_v1_user_metadata_proto_init() } +func file_temporal_api_sdk_v1_user_metadata_proto_init() { + if File_temporal_api_sdk_v1_user_metadata_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_sdk_v1_user_metadata_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*UserMetadata); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_sdk_v1_user_metadata_proto_rawDesc, + NumEnums: 0, + NumMessages: 1, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_sdk_v1_user_metadata_proto_goTypes, + DependencyIndexes: file_temporal_api_sdk_v1_user_metadata_proto_depIdxs, + MessageInfos: file_temporal_api_sdk_v1_user_metadata_proto_msgTypes, + }.Build() + File_temporal_api_sdk_v1_user_metadata_proto = out.File + file_temporal_api_sdk_v1_user_metadata_proto_rawDesc = nil + file_temporal_api_sdk_v1_user_metadata_proto_goTypes = nil + file_temporal_api_sdk_v1_user_metadata_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/sdk/v1/workflow_metadata.go-helpers.pb.go b/vendor/go.temporal.io/api/sdk/v1/workflow_metadata.go-helpers.pb.go new file mode 100644 index 00000000000..091616d9247 --- /dev/null +++ b/vendor/go.temporal.io/api/sdk/v1/workflow_metadata.go-helpers.pb.go @@ -0,0 +1,139 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package sdk + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type WorkflowMetadata to the protobuf v3 wire format +func (val *WorkflowMetadata) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowMetadata from the protobuf v3 wire format +func (val *WorkflowMetadata) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowMetadata) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowMetadata values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowMetadata) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowMetadata + switch t := that.(type) { + case *WorkflowMetadata: + that1 = t + case WorkflowMetadata: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowDefinition to the protobuf v3 wire format +func (val *WorkflowDefinition) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowDefinition from the protobuf v3 wire format +func (val *WorkflowDefinition) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowDefinition) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowDefinition values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowDefinition) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowDefinition + switch t := that.(type) { + case *WorkflowDefinition: + that1 = t + case WorkflowDefinition: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowInteractionDefinition to the protobuf v3 wire format +func (val *WorkflowInteractionDefinition) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowInteractionDefinition from the protobuf v3 wire format +func (val *WorkflowInteractionDefinition) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowInteractionDefinition) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowInteractionDefinition values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowInteractionDefinition) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowInteractionDefinition + switch t := that.(type) { + case *WorkflowInteractionDefinition: + that1 = t + case WorkflowInteractionDefinition: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/sdk/v1/workflow_metadata.pb.go b/vendor/go.temporal.io/api/sdk/v1/workflow_metadata.pb.go new file mode 100644 index 00000000000..b444e4fe95f --- /dev/null +++ b/vendor/go.temporal.io/api/sdk/v1/workflow_metadata.pb.go @@ -0,0 +1,389 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/sdk/v1/workflow_metadata.proto + +package sdk + +import ( + reflect "reflect" + sync "sync" + + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// The name of the query to retrieve this information is `__temporal_workflow_metadata`. +type WorkflowMetadata struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Metadata provided at declaration or creation time. + Definition *WorkflowDefinition `protobuf:"bytes,1,opt,name=definition,proto3" json:"definition,omitempty"` + // Current long-form details of the workflow's state. This is used by user interfaces to show + // long-form text. This text may be formatted by the user interface. + CurrentDetails string `protobuf:"bytes,2,opt,name=current_details,json=currentDetails,proto3" json:"current_details,omitempty"` +} + +func (x *WorkflowMetadata) Reset() { + *x = WorkflowMetadata{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_sdk_v1_workflow_metadata_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowMetadata) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowMetadata) ProtoMessage() {} + +func (x *WorkflowMetadata) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_sdk_v1_workflow_metadata_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowMetadata.ProtoReflect.Descriptor instead. +func (*WorkflowMetadata) Descriptor() ([]byte, []int) { + return file_temporal_api_sdk_v1_workflow_metadata_proto_rawDescGZIP(), []int{0} +} + +func (x *WorkflowMetadata) GetDefinition() *WorkflowDefinition { + if x != nil { + return x.Definition + } + return nil +} + +func (x *WorkflowMetadata) GetCurrentDetails() string { + if x != nil { + return x.CurrentDetails + } + return "" +} + +// (-- api-linter: core::0203::optional=disabled --) +type WorkflowDefinition struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // A name scoped by the task queue that maps to this workflow definition. + // If missing, this workflow is a dynamic workflow. + Type string `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"` + QueryDefinitions []*WorkflowInteractionDefinition `protobuf:"bytes,2,rep,name=query_definitions,json=queryDefinitions,proto3" json:"query_definitions,omitempty"` + SignalDefinitions []*WorkflowInteractionDefinition `protobuf:"bytes,3,rep,name=signal_definitions,json=signalDefinitions,proto3" json:"signal_definitions,omitempty"` + UpdateDefinitions []*WorkflowInteractionDefinition `protobuf:"bytes,4,rep,name=update_definitions,json=updateDefinitions,proto3" json:"update_definitions,omitempty"` +} + +func (x *WorkflowDefinition) Reset() { + *x = WorkflowDefinition{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_sdk_v1_workflow_metadata_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowDefinition) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowDefinition) ProtoMessage() {} + +func (x *WorkflowDefinition) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_sdk_v1_workflow_metadata_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowDefinition.ProtoReflect.Descriptor instead. +func (*WorkflowDefinition) Descriptor() ([]byte, []int) { + return file_temporal_api_sdk_v1_workflow_metadata_proto_rawDescGZIP(), []int{1} +} + +func (x *WorkflowDefinition) GetType() string { + if x != nil { + return x.Type + } + return "" +} + +func (x *WorkflowDefinition) GetQueryDefinitions() []*WorkflowInteractionDefinition { + if x != nil { + return x.QueryDefinitions + } + return nil +} + +func (x *WorkflowDefinition) GetSignalDefinitions() []*WorkflowInteractionDefinition { + if x != nil { + return x.SignalDefinitions + } + return nil +} + +func (x *WorkflowDefinition) GetUpdateDefinitions() []*WorkflowInteractionDefinition { + if x != nil { + return x.UpdateDefinitions + } + return nil +} + +// (-- api-linter: core::0123::resource-annotation=disabled +// +// aip.dev/not-precedent: The `name` field is optional. --) +// +// (-- api-linter: core::0203::optional=disabled --) +type WorkflowInteractionDefinition struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // An optional name for the handler. If missing, it represents + // a dynamic handler that processes any interactions not handled by others. + // There is at most one dynamic handler per workflow and interaction kind. + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // An optional interaction description provided by the application. + // By convention, external tools may interpret its first part, + // i.e., ending with a line break, as a summary of the description. + Description string `protobuf:"bytes,2,opt,name=description,proto3" json:"description,omitempty"` +} + +func (x *WorkflowInteractionDefinition) Reset() { + *x = WorkflowInteractionDefinition{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_sdk_v1_workflow_metadata_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowInteractionDefinition) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowInteractionDefinition) ProtoMessage() {} + +func (x *WorkflowInteractionDefinition) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_sdk_v1_workflow_metadata_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowInteractionDefinition.ProtoReflect.Descriptor instead. +func (*WorkflowInteractionDefinition) Descriptor() ([]byte, []int) { + return file_temporal_api_sdk_v1_workflow_metadata_proto_rawDescGZIP(), []int{2} +} + +func (x *WorkflowInteractionDefinition) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *WorkflowInteractionDefinition) GetDescription() string { + if x != nil { + return x.Description + } + return "" +} + +var File_temporal_api_sdk_v1_workflow_metadata_proto protoreflect.FileDescriptor + +var file_temporal_api_sdk_v1_workflow_metadata_proto_rawDesc = []byte{ + 0x0a, 0x2b, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x73, 0x64, + 0x6b, 0x2f, 0x76, 0x31, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x13, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x22, + 0x8c, 0x01, 0x0a, 0x10, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x12, 0x4b, 0x0a, 0x0a, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x64, + 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2b, 0x0a, + 0x0f, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x44, 0x65, 0x74, + 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0xdf, 0x02, 0x0a, 0x12, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, + 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x63, 0x0a, 0x11, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x64, + 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x32, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, + 0x6b, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x74, + 0x65, 0x72, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x10, 0x71, 0x75, 0x65, 0x72, 0x79, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x65, 0x0a, 0x12, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, + 0x5f, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x32, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, + 0x6e, 0x74, 0x65, 0x72, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x65, 0x0a, 0x12, 0x75, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x5f, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x66, 0x69, + 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x66, + 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x5d, 0x0a, 0x1d, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x61, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x04, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x24, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x42, 0x83, 0x01, 0x0a, 0x16, 0x69, 0x6f, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, + 0x31, 0x42, 0x15, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x1d, 0x67, 0x6f, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x73, 0x64, 0x6b, + 0x2f, 0x76, 0x31, 0x3b, 0x73, 0x64, 0x6b, 0xaa, 0x02, 0x15, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x53, 0x64, 0x6b, 0x2e, 0x56, 0x31, 0xea, 0x02, + 0x18, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, + 0x3a, 0x53, 0x64, 0x6b, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_sdk_v1_workflow_metadata_proto_rawDescOnce sync.Once + file_temporal_api_sdk_v1_workflow_metadata_proto_rawDescData = file_temporal_api_sdk_v1_workflow_metadata_proto_rawDesc +) + +func file_temporal_api_sdk_v1_workflow_metadata_proto_rawDescGZIP() []byte { + file_temporal_api_sdk_v1_workflow_metadata_proto_rawDescOnce.Do(func() { + file_temporal_api_sdk_v1_workflow_metadata_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_sdk_v1_workflow_metadata_proto_rawDescData) + }) + return file_temporal_api_sdk_v1_workflow_metadata_proto_rawDescData +} + +var file_temporal_api_sdk_v1_workflow_metadata_proto_msgTypes = make([]protoimpl.MessageInfo, 3) +var file_temporal_api_sdk_v1_workflow_metadata_proto_goTypes = []any{ + (*WorkflowMetadata)(nil), // 0: temporal.api.sdk.v1.WorkflowMetadata + (*WorkflowDefinition)(nil), // 1: temporal.api.sdk.v1.WorkflowDefinition + (*WorkflowInteractionDefinition)(nil), // 2: temporal.api.sdk.v1.WorkflowInteractionDefinition +} +var file_temporal_api_sdk_v1_workflow_metadata_proto_depIdxs = []int32{ + 1, // 0: temporal.api.sdk.v1.WorkflowMetadata.definition:type_name -> temporal.api.sdk.v1.WorkflowDefinition + 2, // 1: temporal.api.sdk.v1.WorkflowDefinition.query_definitions:type_name -> temporal.api.sdk.v1.WorkflowInteractionDefinition + 2, // 2: temporal.api.sdk.v1.WorkflowDefinition.signal_definitions:type_name -> temporal.api.sdk.v1.WorkflowInteractionDefinition + 2, // 3: temporal.api.sdk.v1.WorkflowDefinition.update_definitions:type_name -> temporal.api.sdk.v1.WorkflowInteractionDefinition + 4, // [4:4] is the sub-list for method output_type + 4, // [4:4] is the sub-list for method input_type + 4, // [4:4] is the sub-list for extension type_name + 4, // [4:4] is the sub-list for extension extendee + 0, // [0:4] is the sub-list for field type_name +} + +func init() { file_temporal_api_sdk_v1_workflow_metadata_proto_init() } +func file_temporal_api_sdk_v1_workflow_metadata_proto_init() { + if File_temporal_api_sdk_v1_workflow_metadata_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_sdk_v1_workflow_metadata_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowMetadata); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_sdk_v1_workflow_metadata_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowDefinition); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_sdk_v1_workflow_metadata_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowInteractionDefinition); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_sdk_v1_workflow_metadata_proto_rawDesc, + NumEnums: 0, + NumMessages: 3, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_sdk_v1_workflow_metadata_proto_goTypes, + DependencyIndexes: file_temporal_api_sdk_v1_workflow_metadata_proto_depIdxs, + MessageInfos: file_temporal_api_sdk_v1_workflow_metadata_proto_msgTypes, + }.Build() + File_temporal_api_sdk_v1_workflow_metadata_proto = out.File + file_temporal_api_sdk_v1_workflow_metadata_proto_rawDesc = nil + file_temporal_api_sdk_v1_workflow_metadata_proto_goTypes = nil + file_temporal_api_sdk_v1_workflow_metadata_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/serviceerror/already_exists.go b/vendor/go.temporal.io/api/serviceerror/already_exists.go new file mode 100644 index 00000000000..49505b8f6f0 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/already_exists.go @@ -0,0 +1,63 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +type ( + // AlreadyExists represents general AlreadyExists gRPC error. + AlreadyExists struct { + Message string + st *status.Status + } +) + +// NewAlreadyExist returns new AlreadyExists error. +func NewAlreadyExist(message string) error { + return &AlreadyExists{ + Message: message, + } +} + +// Error returns string message. +func (e *AlreadyExists) Error() string { + return e.Message +} + +func (e *AlreadyExists) Status() *status.Status { + if e.st != nil { + return e.st + } + + return status.New(codes.AlreadyExists, e.Message) +} + +func newAlreadyExists(st *status.Status) error { + return &AlreadyExists{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/canceled.go b/vendor/go.temporal.io/api/serviceerror/canceled.go new file mode 100644 index 00000000000..f5e5ec54b7c --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/canceled.go @@ -0,0 +1,63 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +type ( + // Canceled represents canceled error. + Canceled struct { + Message string + st *status.Status + } +) + +// NewCanceled returns new Canceled error. +func NewCanceled(message string) error { + return &Canceled{ + Message: message, + } +} + +// Error returns string message. +func (e *Canceled) Error() string { + return e.Message +} + +func (e *Canceled) Status() *status.Status { + if e.st != nil { + return e.st + } + + return status.New(codes.Canceled, e.Message) +} + +func newCanceled(st *status.Status) error { + return &Canceled{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/cancellation_already_requested.go b/vendor/go.temporal.io/api/serviceerror/cancellation_already_requested.go new file mode 100644 index 00000000000..996ccb1812e --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/cancellation_already_requested.go @@ -0,0 +1,69 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/api/errordetails/v1" +) + +type ( + // CancellationAlreadyRequested represents cancellation already requested error. + CancellationAlreadyRequested struct { + Message string + st *status.Status + } +) + +// NewCancellationAlreadyRequested returns new CancellationAlreadyRequested error. +func NewCancellationAlreadyRequested(message string) error { + return &CancellationAlreadyRequested{ + Message: message, + } +} + +// Error returns string message. +func (e *CancellationAlreadyRequested) Error() string { + return e.Message +} + +func (e *CancellationAlreadyRequested) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.AlreadyExists, e.Message) + st, _ = st.WithDetails( + &errordetails.CancellationAlreadyRequestedFailure{}, + ) + return st +} + +func newCancellationAlreadyRequested(st *status.Status) error { + return &CancellationAlreadyRequested{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/client_version_not_supported.go b/vendor/go.temporal.io/api/serviceerror/client_version_not_supported.go new file mode 100644 index 00000000000..b21f8e8aad7 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/client_version_not_supported.go @@ -0,0 +1,84 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "fmt" + + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/api/errordetails/v1" +) + +type ( + // ClientVersionNotSupported represents client version is not supported error. + ClientVersionNotSupported struct { + Message string + ClientVersion string + ClientName string + SupportedVersions string + st *status.Status + } +) + +// NewClientVersionNotSupported returns new ClientVersionNotSupported error. +func NewClientVersionNotSupported(clientVersion, clientName, supportedVersions string) error { + return &ClientVersionNotSupported{ + Message: fmt.Sprintf("Client version %s is not supported. Server supports %s versions: %s", clientVersion, clientName, supportedVersions), + ClientVersion: clientVersion, + ClientName: clientName, + SupportedVersions: supportedVersions, + } +} + +// Error returns string message. +func (e *ClientVersionNotSupported) Error() string { + return e.Message +} + +func (e *ClientVersionNotSupported) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.FailedPrecondition, e.Message) + st, _ = st.WithDetails( + &errordetails.ClientVersionNotSupportedFailure{ + ClientVersion: e.ClientVersion, + ClientName: e.ClientName, + SupportedVersions: e.SupportedVersions, + }, + ) + return st +} + +func newClientVersionNotSupported(st *status.Status, errDetails *errordetails.ClientVersionNotSupportedFailure) error { + return &ClientVersionNotSupported{ + Message: st.Message(), + ClientVersion: errDetails.GetClientVersion(), + ClientName: errDetails.GetClientName(), + SupportedVersions: errDetails.GetSupportedVersions(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/convert.go b/vendor/go.temporal.io/api/serviceerror/convert.go new file mode 100644 index 00000000000..011f3a9b418 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/convert.go @@ -0,0 +1,203 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "context" + "errors" + + spb "google.golang.org/genproto/googleapis/rpc/status" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/api/errordetails/v1" + "go.temporal.io/api/failure/v1" +) + +// ToStatus converts service error to gRPC Status. +// If error is not a service error it returns status with code Unknown. +func ToStatus(err error) *status.Status { + if err == nil { + return status.New(codes.OK, "") + } + + if svcerr, ok := err.(ServiceError); ok { + return svcerr.Status() + } + // err does not implement ServiceError directly, but check if it wraps it. + // This path does more allocation so prefer to return a ServiceError directly if possible. + var svcerr ServiceError + if errors.As(err, &svcerr) { + s := svcerr.Status().Proto() + s.Message = err.Error() // don't lose the wrapped message + return status.FromProto(s) + } + + // Special case for context.DeadlineExceeded and context.Canceled because they can happen in unpredictable places. + if errors.Is(err, context.DeadlineExceeded) { + return status.New(codes.DeadlineExceeded, err.Error()) + } + if errors.Is(err, context.Canceled) { + return status.New(codes.Canceled, err.Error()) + } + + // Internal logic of status.Convert is: + // - if err is already Status or gRPC Status, then just return it (this should never happen though). + // - otherwise returns codes.Unknown with message from err.Error() (this might happen if some generic go error reach to this point). + return status.Convert(err) +} + +// FromStatus converts gRPC Status to service error. +func FromStatus(st *status.Status) error { + if st == nil || st.Code() == codes.OK { + return nil + } + + errDetails := extractErrorDetails(st) + + // Special case: MultiOperation error can have any status code. + if err, ok := errDetails.(*errordetails.MultiOperationExecutionFailure); ok { + errs := make([]error, len(err.Statuses)) + for i, opStatus := range err.Statuses { + errs[i] = FromStatus(status.FromProto(&spb.Status{ + Code: opStatus.Code, + Message: opStatus.Message, + Details: opStatus.Details, + })) + } + return newMultiOperationExecution(st, errs) + } + + // If there was an error during details extraction, for example unknown message type, + // which can happen when new error details are added and getting read by old clients, + // then errDetails will be of type `error` with corresponding error inside. + // This error is ignored and `serviceerror` is built using `st.Code()` only. + switch st.Code() { + case codes.DataLoss: + return newDataLoss(st) + case codes.DeadlineExceeded: + return newDeadlineExceeded(st) + case codes.Canceled: + return newCanceled(st) + case codes.Unavailable: + return newUnavailable(st) + case codes.Unimplemented: + return newUnimplemented(st) + case codes.Unknown: + // Unwrap error message from unknown error. + return errors.New(st.Message()) + case codes.Aborted: + switch errDetails.(type) { + case *failure.MultiOperationExecutionAborted: + return newMultiOperationAborted(st) + default: + // fall through to st.Err() + } + case codes.Internal: + switch errDetails := errDetails.(type) { + case *errordetails.SystemWorkflowFailure: + return newSystemWorkflow(st, errDetails) + default: + return newInternal(st) + } + case codes.NotFound: + switch errDetails := errDetails.(type) { + case *errordetails.NotFoundFailure: + return newNotFound(st, errDetails) + case *errordetails.NamespaceNotFoundFailure: + return newNamespaceNotFound(st, errDetails) + default: + return newNotFound(st, nil) + } + case codes.InvalidArgument: + switch errDetails.(type) { + case *errordetails.QueryFailedFailure: + return newQueryFailed(st) + default: + return newInvalidArgument(st) + } + case codes.ResourceExhausted: + switch errDetails := errDetails.(type) { + case *errordetails.ResourceExhaustedFailure: + return newResourceExhausted(st, errDetails) + default: + return newResourceExhausted(st, nil) + } + case codes.AlreadyExists: + switch errDetails := errDetails.(type) { + case *errordetails.NamespaceAlreadyExistsFailure: + return newNamespaceAlreadyExists(st) + case *errordetails.WorkflowExecutionAlreadyStartedFailure: + return newWorkflowExecutionAlreadyStarted(st, errDetails) + case *errordetails.CancellationAlreadyRequestedFailure: + return newCancellationAlreadyRequested(st) + default: + return newAlreadyExists(st) + } + case codes.FailedPrecondition: + switch errDetails := errDetails.(type) { + case *errordetails.NamespaceNotActiveFailure: + return newNamespaceNotActive(st, errDetails) + case *errordetails.NamespaceInvalidStateFailure: + return newNamespaceInvalidState(st, errDetails) + case *errordetails.ClientVersionNotSupportedFailure: + return newClientVersionNotSupported(st, errDetails) + case *errordetails.ServerVersionNotSupportedFailure: + return newServerVersionNotSupported(st, errDetails) + case *errordetails.WorkflowNotReadyFailure: + return newWorkflowNotReady(st) + default: + return newFailedPrecondition(st) + } + case codes.PermissionDenied: + switch errDetails := errDetails.(type) { + case *errordetails.PermissionDeniedFailure: + return newPermissionDenied(st, errDetails) + default: + return newPermissionDenied(st, nil) + } + case codes.OutOfRange: + switch errDetails := errDetails.(type) { + case *errordetails.NewerBuildExistsFailure: + return newNewerBuildExists(st, errDetails) + default: + // fall through to st.Err() + } + // Unsupported code: + case codes.Unauthenticated: + // fall through to st.Err() + } + + // `st.Code()` has unknown value (should never happen). + // Use standard gRPC error representation "rpc error: code = %s desc = %s". + return st.Err() +} + +func extractErrorDetails(st *status.Status) interface{} { + details := st.Details() + if len(details) > 0 { + return details[0] + } + + return nil +} diff --git a/vendor/go.temporal.io/api/serviceerror/data_loss.go b/vendor/go.temporal.io/api/serviceerror/data_loss.go new file mode 100644 index 00000000000..3443edd520c --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/data_loss.go @@ -0,0 +1,63 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +type ( + // DataLoss represents data loss error. + DataLoss struct { + Message string + st *status.Status + } +) + +// NewDataLoss returns new DataLoss error. +func NewDataLoss(message string) error { + return &DataLoss{ + Message: message, + } +} + +// Error returns string message. +func (e *DataLoss) Error() string { + return e.Message +} + +func (e *DataLoss) Status() *status.Status { + if e.st != nil { + return e.st + } + + return status.New(codes.DataLoss, e.Message) +} + +func newDataLoss(st *status.Status) error { + return &DataLoss{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/deadline_exceeded.go b/vendor/go.temporal.io/api/serviceerror/deadline_exceeded.go new file mode 100644 index 00000000000..286470d4aff --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/deadline_exceeded.go @@ -0,0 +1,63 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +type ( + // DeadlineExceeded represents deadline exceeded error. + DeadlineExceeded struct { + Message string + st *status.Status + } +) + +// NewDeadlineExceeded returns new DeadlineExceeded error. +func NewDeadlineExceeded(message string) error { + return &DeadlineExceeded{ + Message: message, + } +} + +// Error returns string message. +func (e *DeadlineExceeded) Error() string { + return e.Message +} + +func (e *DeadlineExceeded) Status() *status.Status { + if e.st != nil { + return e.st + } + + return status.New(codes.DeadlineExceeded, e.Message) +} + +func newDeadlineExceeded(st *status.Status) error { + return &DeadlineExceeded{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/failed_precondition.go b/vendor/go.temporal.io/api/serviceerror/failed_precondition.go new file mode 100644 index 00000000000..8a5f489f55d --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/failed_precondition.go @@ -0,0 +1,63 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +type ( + // FailedPrecondition represents failed precondition error. + FailedPrecondition struct { + Message string + st *status.Status + } +) + +// NewFailedPrecondition returns new FailedPrecondition error. +func NewFailedPrecondition(message string) error { + return &FailedPrecondition{ + Message: message, + } +} + +// Error returns string message. +func (e *FailedPrecondition) Error() string { + return e.Message +} + +func (e *FailedPrecondition) Status() *status.Status { + if e.st != nil { + return e.st + } + + return status.New(codes.FailedPrecondition, e.Message) +} + +func newFailedPrecondition(st *status.Status) error { + return &FailedPrecondition{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/internal.go b/vendor/go.temporal.io/api/serviceerror/internal.go new file mode 100644 index 00000000000..b700cc23886 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/internal.go @@ -0,0 +1,63 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +type ( + // Internal represents internal error. + Internal struct { + Message string + st *status.Status + } +) + +// NewInternal returns new Internal error. +func NewInternal(message string) error { + return &Internal{ + Message: message, + } +} + +// Error returns string message. +func (e *Internal) Error() string { + return e.Message +} + +func (e *Internal) Status() *status.Status { + if e.st != nil { + return e.st + } + + return status.New(codes.Internal, e.Message) +} + +func newInternal(st *status.Status) error { + return &Internal{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/invalid_argument.go b/vendor/go.temporal.io/api/serviceerror/invalid_argument.go new file mode 100644 index 00000000000..3dc67422161 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/invalid_argument.go @@ -0,0 +1,63 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +type ( + // InvalidArgument represents invalid argument error. + InvalidArgument struct { + Message string + st *status.Status + } +) + +// NewInvalidArgument returns new InvalidArgument error. +func NewInvalidArgument(message string) error { + return &InvalidArgument{ + Message: message, + } +} + +// Error returns string message. +func (e *InvalidArgument) Error() string { + return e.Message +} + +func (e *InvalidArgument) Status() *status.Status { + if e.st != nil { + return e.st + } + + return status.New(codes.InvalidArgument, e.Message) +} + +func newInvalidArgument(st *status.Status) error { + return &InvalidArgument{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/multi_op.go b/vendor/go.temporal.io/api/serviceerror/multi_op.go new file mode 100644 index 00000000000..3f8e9788596 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/multi_op.go @@ -0,0 +1,94 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "errors" + + "go.temporal.io/api/errordetails/v1" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +// MultiOperationExecution represents a MultiOperationExecution error. +type MultiOperationExecution struct { + Message string + errs []error + st *status.Status +} + +// NewMultiOperationExecution returns a new MultiOperationExecution error. +func NewMultiOperationExecution(message string, errs []error) error { + return &MultiOperationExecution{Message: message, errs: errs} +} + +// Error returns string message. +func (e *MultiOperationExecution) Error() string { + return e.Message +} + +func (e *MultiOperationExecution) OperationErrors() []error { + return e.errs +} + +func (e *MultiOperationExecution) Status() *status.Status { + var code *codes.Code + failure := &errordetails.MultiOperationExecutionFailure{ + Statuses: make([]*errordetails.MultiOperationExecutionFailure_OperationStatus, len(e.errs)), + } + + var abortedErr *MultiOperationAborted + for i, err := range e.errs { + st := ToStatus(err) + + // the first non-OK and non-Aborted code becomes the code for the entire Status + if code == nil && st.Code() != codes.OK && !errors.As(err, &abortedErr) { + c := st.Code() + code = &c + } + + failure.Statuses[i] = &errordetails.MultiOperationExecutionFailure_OperationStatus{ + Code: int32(st.Code()), + Message: st.Message(), + Details: st.Proto().Details, + } + } + + // this should never happen, but it's better to set it to `Aborted` than to panic + if code == nil { + c := codes.Aborted + code = &c + } + + st := status.New(*code, e.Error()) + st, _ = st.WithDetails(failure) + return st +} + +func newMultiOperationExecution(st *status.Status, errs []error) error { + return &MultiOperationExecution{ + Message: st.Message(), + errs: errs, + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/multi_op_aborted.go b/vendor/go.temporal.io/api/serviceerror/multi_op_aborted.go new file mode 100644 index 00000000000..1374da1580d --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/multi_op_aborted.go @@ -0,0 +1,64 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + failurepb "go.temporal.io/api/failure/v1" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +// MultiOperationAborted represents an aborted operation from a MultiOperationExecution. +type MultiOperationAborted struct { + Message string + st *status.Status +} + +// NewMultiOperationAborted returns MultiOperationAborted. +func NewMultiOperationAborted(message string) error { + return &MultiOperationAborted{ + Message: message, + } +} + +// Error returns string message. +func (e MultiOperationAborted) Error() string { + return e.Message +} + +func (e MultiOperationAborted) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.Aborted, e.Error()) + st, _ = st.WithDetails(&failurepb.MultiOperationExecutionAborted{}) + return st +} + +func newMultiOperationAborted(st *status.Status) error { + return &MultiOperationAborted{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/namespace_already_exists.go b/vendor/go.temporal.io/api/serviceerror/namespace_already_exists.go new file mode 100644 index 00000000000..790d6d58fc1 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/namespace_already_exists.go @@ -0,0 +1,69 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/api/errordetails/v1" +) + +type ( + // NamespaceAlreadyExists represents namespace already exists error. + NamespaceAlreadyExists struct { + Message string + st *status.Status + } +) + +// NewNamespaceAlreadyExists returns new NamespaceAlreadyExists error. +func NewNamespaceAlreadyExists(message string) error { + return &NamespaceAlreadyExists{ + Message: message, + } +} + +// Error returns string message. +func (e *NamespaceAlreadyExists) Error() string { + return e.Message +} + +func (e *NamespaceAlreadyExists) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.AlreadyExists, e.Message) + st, _ = st.WithDetails( + &errordetails.NamespaceAlreadyExistsFailure{}, + ) + return st +} + +func newNamespaceAlreadyExists(st *status.Status) error { + return &NamespaceAlreadyExists{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/namespace_invalid_state.go b/vendor/go.temporal.io/api/serviceerror/namespace_invalid_state.go new file mode 100644 index 00000000000..4d84bedd6c6 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/namespace_invalid_state.go @@ -0,0 +1,94 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "fmt" + "strings" + + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/api/errordetails/v1" +) + +type ( + // NamespaceInvalidState represents namespace not active error. + NamespaceInvalidState struct { + Message string + Namespace string + State enumspb.NamespaceState + AllowedStates []enumspb.NamespaceState + st *status.Status + } +) + +// NewNamespaceInvalidState returns new NamespaceInvalidState error. +func NewNamespaceInvalidState(namespace string, state enumspb.NamespaceState, allowedStates []enumspb.NamespaceState) error { + var allowedStatesStr []string + for _, allowedState := range allowedStates { + allowedStatesStr = append(allowedStatesStr, allowedState.String()) + } + return &NamespaceInvalidState{ + Message: fmt.Sprintf( + "Namespace has invalid state: %s. Must be %s.", + state, + strings.Join(allowedStatesStr, " or "), + ), + Namespace: namespace, + State: state, + AllowedStates: allowedStates, + } +} + +// Error returns string message. +func (e *NamespaceInvalidState) Error() string { + return e.Message +} + +func (e *NamespaceInvalidState) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.FailedPrecondition, e.Message) + st, _ = st.WithDetails( + &errordetails.NamespaceInvalidStateFailure{ + Namespace: e.Namespace, + State: e.State, + AllowedStates: e.AllowedStates, + }, + ) + return st +} + +func newNamespaceInvalidState(st *status.Status, errDetails *errordetails.NamespaceInvalidStateFailure) error { + return &NamespaceInvalidState{ + Message: st.Message(), + Namespace: errDetails.GetNamespace(), + State: errDetails.GetState(), + AllowedStates: errDetails.GetAllowedStates(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/namespace_not_active.go b/vendor/go.temporal.io/api/serviceerror/namespace_not_active.go new file mode 100644 index 00000000000..579e4c676ba --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/namespace_not_active.go @@ -0,0 +1,89 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "fmt" + + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/api/errordetails/v1" +) + +type ( + // NamespaceNotActive represents namespace not active error. + NamespaceNotActive struct { + Message string + Namespace string + CurrentCluster string + ActiveCluster string + st *status.Status + } +) + +// NewNamespaceNotActive returns new NamespaceNotActive error. +func NewNamespaceNotActive(namespace, currentCluster, activeCluster string) error { + return &NamespaceNotActive{ + Message: fmt.Sprintf( + "Namespace: %s is active in cluster: %s, while current cluster %s is a standby cluster.", + namespace, + activeCluster, + currentCluster, + ), + Namespace: namespace, + CurrentCluster: currentCluster, + ActiveCluster: activeCluster, + } +} + +// Error returns string message. +func (e *NamespaceNotActive) Error() string { + return e.Message +} + +func (e *NamespaceNotActive) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.FailedPrecondition, e.Message) + st, _ = st.WithDetails( + &errordetails.NamespaceNotActiveFailure{ + Namespace: e.Namespace, + CurrentCluster: e.CurrentCluster, + ActiveCluster: e.ActiveCluster, + }, + ) + return st +} + +func newNamespaceNotActive(st *status.Status, errDetails *errordetails.NamespaceNotActiveFailure) error { + return &NamespaceNotActive{ + Message: st.Message(), + Namespace: errDetails.GetNamespace(), + CurrentCluster: errDetails.GetCurrentCluster(), + ActiveCluster: errDetails.GetActiveCluster(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/namespace_not_found.go b/vendor/go.temporal.io/api/serviceerror/namespace_not_found.go new file mode 100644 index 00000000000..f80a4f84331 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/namespace_not_found.go @@ -0,0 +1,79 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "fmt" + + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/api/errordetails/v1" +) + +type ( + // NamespaceNotFound represents namespace not found error. + NamespaceNotFound struct { + Message string + Namespace string + st *status.Status + } +) + +// NewNamespaceNotFound returns new NamespaceNotFound error. +func NewNamespaceNotFound(namespace string) error { + return &NamespaceNotFound{ + Message: fmt.Sprintf( + "Namespace %s is not found.", + namespace, + ), + Namespace: namespace, + } +} + +// Error returns string message. +func (e *NamespaceNotFound) Error() string { + return e.Message +} + +func (e *NamespaceNotFound) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.NotFound, e.Message) + st, _ = st.WithDetails( + &errordetails.NamespaceNotFoundFailure{ + Namespace: e.Namespace, + }, + ) + return st +} + +func newNamespaceNotFound(st *status.Status, errDetails *errordetails.NamespaceNotFoundFailure) error { + return &NamespaceNotFound{ + Message: st.Message(), + Namespace: errDetails.GetNamespace(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/newer_build_exists.go b/vendor/go.temporal.io/api/serviceerror/newer_build_exists.go new file mode 100644 index 00000000000..caf43156397 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/newer_build_exists.go @@ -0,0 +1,77 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "fmt" + + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/api/errordetails/v1" +) + +type ( + // NewerBuildExists is returned to a poll request from a build that has been superceded by + // a newer build in versioning metadata. + NewerBuildExists struct { + Message string + DefaultBuildID string + st *status.Status + } +) + +// NewNewerBuildExists returns new NewerBuildExists error. +func NewNewerBuildExists(defaultBuildID string) error { + return &NewerBuildExists{ + Message: fmt.Sprintf("Task queue has a newer compatible build: %q", defaultBuildID), + DefaultBuildID: defaultBuildID, + } +} + +// Error returns string message. +func (e *NewerBuildExists) Error() string { + return e.Message +} + +func (e *NewerBuildExists) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.OutOfRange, e.Message) + st, _ = st.WithDetails( + &errordetails.NewerBuildExistsFailure{ + DefaultBuildId: e.DefaultBuildID, + }, + ) + return st +} + +func newNewerBuildExists(st *status.Status, errDetails *errordetails.NewerBuildExistsFailure) error { + return &NewerBuildExists{ + Message: st.Message(), + DefaultBuildID: errDetails.GetDefaultBuildId(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/not_found.go b/vendor/go.temporal.io/api/serviceerror/not_found.go new file mode 100644 index 00000000000..031f73243c9 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/not_found.go @@ -0,0 +1,76 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/api/errordetails/v1" +) + +type ( + // NotFound represents not found error. + NotFound struct { + Message string + CurrentCluster string + ActiveCluster string + st *status.Status + } +) + +// NewNotFound returns new NotFound error. +func NewNotFound(message string) error { + return &NotFound{ + Message: message, + } +} + +// Error returns string message. +func (e *NotFound) Error() string { + return e.Message +} + +func (e *NotFound) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.NotFound, e.Message) + st, _ = st.WithDetails( + &errordetails.NotFoundFailure{ + CurrentCluster: e.CurrentCluster, + ActiveCluster: e.ActiveCluster, + }, + ) + return st +} + +func newNotFound(st *status.Status, errDetails *errordetails.NotFoundFailure) error { + return &NotFound{ + Message: st.Message(), + CurrentCluster: errDetails.GetCurrentCluster(), + ActiveCluster: errDetails.GetActiveCluster(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/permission_denied.go b/vendor/go.temporal.io/api/serviceerror/permission_denied.go new file mode 100644 index 00000000000..006a2ad50a1 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/permission_denied.go @@ -0,0 +1,73 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "go.temporal.io/api/errordetails/v1" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +type ( + // PermissionDenied represents permission denied error. + PermissionDenied struct { + Message string + Reason string + st *status.Status + } +) + +// NewPermissionDenied returns new PermissionDenied error. +func NewPermissionDenied(message, reason string) error { + return &PermissionDenied{ + Message: message, + Reason: reason, + } +} + +// Error returns string message. +func (e *PermissionDenied) Error() string { + return e.Message +} + +func (e *PermissionDenied) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.PermissionDenied, e.Message) + st, _ = st.WithDetails( + &errordetails.PermissionDeniedFailure{ + Reason: e.Reason, + }, + ) + return st +} + +func newPermissionDenied(st *status.Status, errDetails *errordetails.PermissionDeniedFailure) error { + return &PermissionDenied{ + Message: st.Message(), + Reason: errDetails.GetReason(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/query_failed.go b/vendor/go.temporal.io/api/serviceerror/query_failed.go new file mode 100644 index 00000000000..04ae251927c --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/query_failed.go @@ -0,0 +1,69 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/api/errordetails/v1" +) + +type ( + // QueryFailed represents query failed error. + QueryFailed struct { + Message string + st *status.Status + } +) + +// NewQueryFailed returns new QueryFailed error. +func NewQueryFailed(message string) error { + return &QueryFailed{ + Message: message, + } +} + +// Error returns string message. +func (e *QueryFailed) Error() string { + return e.Message +} + +func (e *QueryFailed) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.InvalidArgument, e.Message) + st, _ = st.WithDetails( + &errordetails.QueryFailedFailure{}, + ) + return st +} + +func newQueryFailed(st *status.Status) error { + return &QueryFailed{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/resource_exhausted.go b/vendor/go.temporal.io/api/serviceerror/resource_exhausted.go new file mode 100644 index 00000000000..35ede3335f0 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/resource_exhausted.go @@ -0,0 +1,78 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/api/errordetails/v1" +) + +type ( + // ResourceExhausted represents resource exhausted error. + ResourceExhausted struct { + Cause enumspb.ResourceExhaustedCause + Scope enumspb.ResourceExhaustedScope + Message string + st *status.Status + } +) + +// NewResourceExhausted returns new ResourceExhausted error. +func NewResourceExhausted(cause enumspb.ResourceExhaustedCause, message string) error { + return &ResourceExhausted{ + Cause: cause, + Message: message, + } +} + +// Error returns string message. +func (e *ResourceExhausted) Error() string { + return e.Message +} + +func (e *ResourceExhausted) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.ResourceExhausted, e.Message) + st, _ = st.WithDetails( + &errordetails.ResourceExhaustedFailure{ + Cause: e.Cause, + Scope: e.Scope, + }, + ) + return st +} + +func newResourceExhausted(st *status.Status, errDetails *errordetails.ResourceExhaustedFailure) error { + return &ResourceExhausted{ + Cause: errDetails.GetCause(), + Scope: errDetails.GetScope(), + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/server_version_not_supported.go b/vendor/go.temporal.io/api/serviceerror/server_version_not_supported.go new file mode 100644 index 00000000000..85b5b4e188e --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/server_version_not_supported.go @@ -0,0 +1,80 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "fmt" + + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/api/errordetails/v1" +) + +type ( + // ServerVersionNotSupported represents client version is not supported error. + ServerVersionNotSupported struct { + Message string + ServerVersion string + ClientSupportedServerVersions string + st *status.Status + } +) + +// NewServerVersionNotSupported returns new ServerVersionNotSupported error. +func NewServerVersionNotSupported(serverVersion, supportedVersions string) error { + return &ServerVersionNotSupported{ + Message: fmt.Sprintf("Server version %s is not supported. Client supports server versions: %s", serverVersion, supportedVersions), + ServerVersion: serverVersion, + ClientSupportedServerVersions: supportedVersions, + } +} + +// Error returns string message. +func (e *ServerVersionNotSupported) Error() string { + return e.Message +} + +func (e *ServerVersionNotSupported) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.FailedPrecondition, e.Message) + st, _ = st.WithDetails( + &errordetails.ServerVersionNotSupportedFailure{ + ServerVersion: e.ServerVersion, + ClientSupportedServerVersions: e.ClientSupportedServerVersions, + }, + ) + return st +} + +func newServerVersionNotSupported(st *status.Status, errDetails *errordetails.ServerVersionNotSupportedFailure) error { + return &ServerVersionNotSupported{ + Message: st.Message(), + ServerVersion: errDetails.GetServerVersion(), + ClientSupportedServerVersions: errDetails.GetClientSupportedServerVersions(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/serviceerror.go b/vendor/go.temporal.io/api/serviceerror/serviceerror.go new file mode 100644 index 00000000000..509cd2d325f --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/serviceerror.go @@ -0,0 +1,32 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import "google.golang.org/grpc/status" + +type ( + ServiceError interface { + error + Status() *status.Status + } +) diff --git a/vendor/go.temporal.io/api/serviceerror/system_workflow.go b/vendor/go.temporal.io/api/serviceerror/system_workflow.go new file mode 100644 index 00000000000..d164d4beea5 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/system_workflow.go @@ -0,0 +1,80 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "fmt" + + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/api/common/v1" + "go.temporal.io/api/errordetails/v1" +) + +type ( + // SystemWorkflow represents an error that happens during execution of the underlying system workflow + SystemWorkflow struct { + WorkflowExecution *common.WorkflowExecution + WorkflowError string + st *status.Status + } +) + +// NewSystemWorkflow returns new SystemWorkflow error. +func NewSystemWorkflow(workflowExecution *common.WorkflowExecution, workflowError error) error { + return &SystemWorkflow{ + WorkflowExecution: workflowExecution, + WorkflowError: workflowError.Error(), + } +} + +// Error returns string message. +func (e *SystemWorkflow) Error() string { + execution := e.WorkflowExecution + return fmt.Sprintf("System Workflow with WorkflowId %s and RunId %s returned an error: %s", + execution.WorkflowId, execution.RunId, e.WorkflowError) +} + +func (e *SystemWorkflow) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.Internal, e.Error()) + st, _ = st.WithDetails( + &errordetails.SystemWorkflowFailure{ + WorkflowExecution: e.WorkflowExecution, + WorkflowError: e.WorkflowError, + }, + ) + return st +} + +func newSystemWorkflow(st *status.Status, errDetails *errordetails.SystemWorkflowFailure) error { + return &SystemWorkflow{ + WorkflowExecution: errDetails.WorkflowExecution, + WorkflowError: errDetails.WorkflowError, + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/unavailable.go b/vendor/go.temporal.io/api/serviceerror/unavailable.go new file mode 100644 index 00000000000..fee56c0356d --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/unavailable.go @@ -0,0 +1,63 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +type ( + // Unavailable represents unavailable error. + Unavailable struct { + Message string + st *status.Status + } +) + +// NewUnavailable returns new Unavailable error. +func NewUnavailable(message string) error { + return &Unavailable{ + Message: message, + } +} + +// Error returns string message. +func (e *Unavailable) Error() string { + return e.Message +} + +func (e *Unavailable) Status() *status.Status { + if e.st != nil { + return e.st + } + + return status.New(codes.Unavailable, e.Message) +} + +func newUnavailable(st *status.Status) error { + return &Unavailable{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/unimplemented.go b/vendor/go.temporal.io/api/serviceerror/unimplemented.go new file mode 100644 index 00000000000..786b2e7624a --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/unimplemented.go @@ -0,0 +1,63 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +type ( + // Unimplemented represents unimplemented error. + Unimplemented struct { + Message string + st *status.Status + } +) + +// NewUnimplemented returns new Unimplemented error. +func NewUnimplemented(message string) error { + return &Unimplemented{ + Message: message, + } +} + +// Error returns string message. +func (e *Unimplemented) Error() string { + return e.Message +} + +func (e *Unimplemented) Status() *status.Status { + if e.st != nil { + return e.st + } + + return status.New(codes.Unimplemented, e.Message) +} + +func newUnimplemented(st *status.Status) error { + return &Unimplemented{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/workflow_execution_already_started.go b/vendor/go.temporal.io/api/serviceerror/workflow_execution_already_started.go new file mode 100644 index 00000000000..ac72c5082cd --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/workflow_execution_already_started.go @@ -0,0 +1,78 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/api/errordetails/v1" +) + +type ( + // WorkflowExecutionAlreadyStarted represents workflow execution already started error. + WorkflowExecutionAlreadyStarted struct { + Message string + StartRequestId string + RunId string + st *status.Status + } +) + +// NewWorkflowExecutionAlreadyStarted returns new WorkflowExecutionAlreadyStarted error. +func NewWorkflowExecutionAlreadyStarted(message, startRequestId, runId string) error { + return &WorkflowExecutionAlreadyStarted{ + Message: message, + StartRequestId: startRequestId, + RunId: runId, + } +} + +// Error returns string message. +func (e *WorkflowExecutionAlreadyStarted) Error() string { + return e.Message +} + +func (e *WorkflowExecutionAlreadyStarted) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.AlreadyExists, e.Message) + st, _ = st.WithDetails( + &errordetails.WorkflowExecutionAlreadyStartedFailure{ + StartRequestId: e.StartRequestId, + RunId: e.RunId, + }, + ) + return st +} + +func newWorkflowExecutionAlreadyStarted(st *status.Status, errDetails *errordetails.WorkflowExecutionAlreadyStartedFailure) error { + return &WorkflowExecutionAlreadyStarted{ + Message: st.Message(), + StartRequestId: errDetails.GetStartRequestId(), + RunId: errDetails.GetRunId(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/serviceerror/workflow_not_ready.go b/vendor/go.temporal.io/api/serviceerror/workflow_not_ready.go new file mode 100644 index 00000000000..b69472d7263 --- /dev/null +++ b/vendor/go.temporal.io/api/serviceerror/workflow_not_ready.go @@ -0,0 +1,69 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serviceerror + +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/api/errordetails/v1" +) + +type ( + // WorkflowNotReady represents workflow state is not ready to handle the request error. + WorkflowNotReady struct { + Message string + st *status.Status + } +) + +// NewWorkflowNotReady returns new WorkflowNotReady +func NewWorkflowNotReady(message string) error { + return &WorkflowNotReady{ + Message: message, + } +} + +// Error returns string message. +func (e *WorkflowNotReady) Error() string { + return e.Message +} + +func (e *WorkflowNotReady) Status() *status.Status { + if e.st != nil { + return e.st + } + + st := status.New(codes.FailedPrecondition, e.Message) + st, _ = st.WithDetails( + &errordetails.WorkflowNotReadyFailure{}, + ) + return st +} + +func newWorkflowNotReady(st *status.Status) error { + return &WorkflowNotReady{ + Message: st.Message(), + st: st, + } +} diff --git a/vendor/go.temporal.io/api/taskqueue/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/taskqueue/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..6da0af79b16 --- /dev/null +++ b/vendor/go.temporal.io/api/taskqueue/v1/message.go-helpers.pb.go @@ -0,0 +1,731 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package taskqueue + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type TaskQueue to the protobuf v3 wire format +func (val *TaskQueue) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TaskQueue from the protobuf v3 wire format +func (val *TaskQueue) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TaskQueue) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TaskQueue values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TaskQueue) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TaskQueue + switch t := that.(type) { + case *TaskQueue: + that1 = t + case TaskQueue: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TaskQueueMetadata to the protobuf v3 wire format +func (val *TaskQueueMetadata) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TaskQueueMetadata from the protobuf v3 wire format +func (val *TaskQueueMetadata) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TaskQueueMetadata) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TaskQueueMetadata values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TaskQueueMetadata) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TaskQueueMetadata + switch t := that.(type) { + case *TaskQueueMetadata: + that1 = t + case TaskQueueMetadata: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TaskQueueVersionSelection to the protobuf v3 wire format +func (val *TaskQueueVersionSelection) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TaskQueueVersionSelection from the protobuf v3 wire format +func (val *TaskQueueVersionSelection) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TaskQueueVersionSelection) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TaskQueueVersionSelection values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TaskQueueVersionSelection) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TaskQueueVersionSelection + switch t := that.(type) { + case *TaskQueueVersionSelection: + that1 = t + case TaskQueueVersionSelection: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TaskQueueVersionInfo to the protobuf v3 wire format +func (val *TaskQueueVersionInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TaskQueueVersionInfo from the protobuf v3 wire format +func (val *TaskQueueVersionInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TaskQueueVersionInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TaskQueueVersionInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TaskQueueVersionInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TaskQueueVersionInfo + switch t := that.(type) { + case *TaskQueueVersionInfo: + that1 = t + case TaskQueueVersionInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TaskQueueTypeInfo to the protobuf v3 wire format +func (val *TaskQueueTypeInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TaskQueueTypeInfo from the protobuf v3 wire format +func (val *TaskQueueTypeInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TaskQueueTypeInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TaskQueueTypeInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TaskQueueTypeInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TaskQueueTypeInfo + switch t := that.(type) { + case *TaskQueueTypeInfo: + that1 = t + case TaskQueueTypeInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TaskQueueStats to the protobuf v3 wire format +func (val *TaskQueueStats) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TaskQueueStats from the protobuf v3 wire format +func (val *TaskQueueStats) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TaskQueueStats) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TaskQueueStats values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TaskQueueStats) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TaskQueueStats + switch t := that.(type) { + case *TaskQueueStats: + that1 = t + case TaskQueueStats: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TaskQueueStatus to the protobuf v3 wire format +func (val *TaskQueueStatus) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TaskQueueStatus from the protobuf v3 wire format +func (val *TaskQueueStatus) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TaskQueueStatus) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TaskQueueStatus values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TaskQueueStatus) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TaskQueueStatus + switch t := that.(type) { + case *TaskQueueStatus: + that1 = t + case TaskQueueStatus: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TaskIdBlock to the protobuf v3 wire format +func (val *TaskIdBlock) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TaskIdBlock from the protobuf v3 wire format +func (val *TaskIdBlock) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TaskIdBlock) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TaskIdBlock values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TaskIdBlock) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TaskIdBlock + switch t := that.(type) { + case *TaskIdBlock: + that1 = t + case TaskIdBlock: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TaskQueuePartitionMetadata to the protobuf v3 wire format +func (val *TaskQueuePartitionMetadata) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TaskQueuePartitionMetadata from the protobuf v3 wire format +func (val *TaskQueuePartitionMetadata) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TaskQueuePartitionMetadata) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TaskQueuePartitionMetadata values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TaskQueuePartitionMetadata) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TaskQueuePartitionMetadata + switch t := that.(type) { + case *TaskQueuePartitionMetadata: + that1 = t + case TaskQueuePartitionMetadata: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PollerInfo to the protobuf v3 wire format +func (val *PollerInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PollerInfo from the protobuf v3 wire format +func (val *PollerInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PollerInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PollerInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PollerInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PollerInfo + switch t := that.(type) { + case *PollerInfo: + that1 = t + case PollerInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StickyExecutionAttributes to the protobuf v3 wire format +func (val *StickyExecutionAttributes) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StickyExecutionAttributes from the protobuf v3 wire format +func (val *StickyExecutionAttributes) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StickyExecutionAttributes) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StickyExecutionAttributes values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StickyExecutionAttributes) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StickyExecutionAttributes + switch t := that.(type) { + case *StickyExecutionAttributes: + that1 = t + case StickyExecutionAttributes: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CompatibleVersionSet to the protobuf v3 wire format +func (val *CompatibleVersionSet) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CompatibleVersionSet from the protobuf v3 wire format +func (val *CompatibleVersionSet) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CompatibleVersionSet) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CompatibleVersionSet values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CompatibleVersionSet) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CompatibleVersionSet + switch t := that.(type) { + case *CompatibleVersionSet: + that1 = t + case CompatibleVersionSet: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TaskQueueReachability to the protobuf v3 wire format +func (val *TaskQueueReachability) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TaskQueueReachability from the protobuf v3 wire format +func (val *TaskQueueReachability) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TaskQueueReachability) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TaskQueueReachability values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TaskQueueReachability) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TaskQueueReachability + switch t := that.(type) { + case *TaskQueueReachability: + that1 = t + case TaskQueueReachability: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type BuildIdReachability to the protobuf v3 wire format +func (val *BuildIdReachability) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type BuildIdReachability from the protobuf v3 wire format +func (val *BuildIdReachability) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *BuildIdReachability) Size() int { + return proto.Size(val) +} + +// Equal returns whether two BuildIdReachability values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *BuildIdReachability) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *BuildIdReachability + switch t := that.(type) { + case *BuildIdReachability: + that1 = t + case BuildIdReachability: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RampByPercentage to the protobuf v3 wire format +func (val *RampByPercentage) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RampByPercentage from the protobuf v3 wire format +func (val *RampByPercentage) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RampByPercentage) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RampByPercentage values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RampByPercentage) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RampByPercentage + switch t := that.(type) { + case *RampByPercentage: + that1 = t + case RampByPercentage: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type BuildIdAssignmentRule to the protobuf v3 wire format +func (val *BuildIdAssignmentRule) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type BuildIdAssignmentRule from the protobuf v3 wire format +func (val *BuildIdAssignmentRule) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *BuildIdAssignmentRule) Size() int { + return proto.Size(val) +} + +// Equal returns whether two BuildIdAssignmentRule values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *BuildIdAssignmentRule) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *BuildIdAssignmentRule + switch t := that.(type) { + case *BuildIdAssignmentRule: + that1 = t + case BuildIdAssignmentRule: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CompatibleBuildIdRedirectRule to the protobuf v3 wire format +func (val *CompatibleBuildIdRedirectRule) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CompatibleBuildIdRedirectRule from the protobuf v3 wire format +func (val *CompatibleBuildIdRedirectRule) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CompatibleBuildIdRedirectRule) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CompatibleBuildIdRedirectRule values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CompatibleBuildIdRedirectRule) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CompatibleBuildIdRedirectRule + switch t := that.(type) { + case *CompatibleBuildIdRedirectRule: + that1 = t + case CompatibleBuildIdRedirectRule: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TimestampedBuildIdAssignmentRule to the protobuf v3 wire format +func (val *TimestampedBuildIdAssignmentRule) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TimestampedBuildIdAssignmentRule from the protobuf v3 wire format +func (val *TimestampedBuildIdAssignmentRule) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TimestampedBuildIdAssignmentRule) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TimestampedBuildIdAssignmentRule values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TimestampedBuildIdAssignmentRule) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TimestampedBuildIdAssignmentRule + switch t := that.(type) { + case *TimestampedBuildIdAssignmentRule: + that1 = t + case TimestampedBuildIdAssignmentRule: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TimestampedCompatibleBuildIdRedirectRule to the protobuf v3 wire format +func (val *TimestampedCompatibleBuildIdRedirectRule) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TimestampedCompatibleBuildIdRedirectRule from the protobuf v3 wire format +func (val *TimestampedCompatibleBuildIdRedirectRule) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TimestampedCompatibleBuildIdRedirectRule) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TimestampedCompatibleBuildIdRedirectRule values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TimestampedCompatibleBuildIdRedirectRule) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TimestampedCompatibleBuildIdRedirectRule + switch t := that.(type) { + case *TimestampedCompatibleBuildIdRedirectRule: + that1 = t + case TimestampedCompatibleBuildIdRedirectRule: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/taskqueue/v1/message.pb.go b/vendor/go.temporal.io/api/taskqueue/v1/message.pb.go new file mode 100644 index 00000000000..04f88117a29 --- /dev/null +++ b/vendor/go.temporal.io/api/taskqueue/v1/message.pb.go @@ -0,0 +1,1824 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/taskqueue/v1/message.proto + +package taskqueue + +import ( + reflect "reflect" + sync "sync" + + v11 "go.temporal.io/api/common/v1" + v1 "go.temporal.io/api/enums/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + durationpb "google.golang.org/protobuf/types/known/durationpb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" + wrapperspb "google.golang.org/protobuf/types/known/wrapperspb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// See https://docs.temporal.io/docs/concepts/task-queues/ +type TaskQueue struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // Default: TASK_QUEUE_KIND_NORMAL. + Kind v1.TaskQueueKind `protobuf:"varint,2,opt,name=kind,proto3,enum=temporal.api.enums.v1.TaskQueueKind" json:"kind,omitempty"` + // Iff kind == TASK_QUEUE_KIND_STICKY, then this field contains the name of + // the normal task queue that the sticky worker is running on. + NormalName string `protobuf:"bytes,3,opt,name=normal_name,json=normalName,proto3" json:"normal_name,omitempty"` +} + +func (x *TaskQueue) Reset() { + *x = TaskQueue{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TaskQueue) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskQueue) ProtoMessage() {} + +func (x *TaskQueue) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TaskQueue.ProtoReflect.Descriptor instead. +func (*TaskQueue) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *TaskQueue) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *TaskQueue) GetKind() v1.TaskQueueKind { + if x != nil { + return x.Kind + } + return v1.TaskQueueKind(0) +} + +func (x *TaskQueue) GetNormalName() string { + if x != nil { + return x.NormalName + } + return "" +} + +// Only applies to activity task queues +type TaskQueueMetadata struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Allows throttling dispatch of tasks from this queue + MaxTasksPerSecond *wrapperspb.DoubleValue `protobuf:"bytes,1,opt,name=max_tasks_per_second,json=maxTasksPerSecond,proto3" json:"max_tasks_per_second,omitempty"` +} + +func (x *TaskQueueMetadata) Reset() { + *x = TaskQueueMetadata{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TaskQueueMetadata) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskQueueMetadata) ProtoMessage() {} + +func (x *TaskQueueMetadata) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TaskQueueMetadata.ProtoReflect.Descriptor instead. +func (*TaskQueueMetadata) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *TaskQueueMetadata) GetMaxTasksPerSecond() *wrapperspb.DoubleValue { + if x != nil { + return x.MaxTasksPerSecond + } + return nil +} + +// Used for specifying versions the caller is interested in. +type TaskQueueVersionSelection struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Include specific Build IDs. + BuildIds []string `protobuf:"bytes,1,rep,name=build_ids,json=buildIds,proto3" json:"build_ids,omitempty"` + // Include the unversioned queue. + Unversioned bool `protobuf:"varint,2,opt,name=unversioned,proto3" json:"unversioned,omitempty"` + // Include all active versions. A version is considered active if it has had new + // tasks or polls recently. + AllActive bool `protobuf:"varint,3,opt,name=all_active,json=allActive,proto3" json:"all_active,omitempty"` +} + +func (x *TaskQueueVersionSelection) Reset() { + *x = TaskQueueVersionSelection{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TaskQueueVersionSelection) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskQueueVersionSelection) ProtoMessage() {} + +func (x *TaskQueueVersionSelection) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TaskQueueVersionSelection.ProtoReflect.Descriptor instead. +func (*TaskQueueVersionSelection) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *TaskQueueVersionSelection) GetBuildIds() []string { + if x != nil { + return x.BuildIds + } + return nil +} + +func (x *TaskQueueVersionSelection) GetUnversioned() bool { + if x != nil { + return x.Unversioned + } + return false +} + +func (x *TaskQueueVersionSelection) GetAllActive() bool { + if x != nil { + return x.AllActive + } + return false +} + +type TaskQueueVersionInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Task Queue info per Task Type. Key is the numerical value of the temporal.api.enums.v1.TaskQueueType enum. + TypesInfo map[int32]*TaskQueueTypeInfo `protobuf:"bytes,1,rep,name=types_info,json=typesInfo,proto3" json:"types_info,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // Task Reachability is eventually consistent; there may be a delay until it converges to the most + // accurate value but it is designed in a way to take the more conservative side until it converges. + // For example REACHABLE is more conservative than CLOSED_WORKFLOWS_ONLY. + // + // Note: future activities who inherit their workflow's Build ID but not its Task Queue will not be + // accounted for reachability as server cannot know if they'll happen as they do not use + // assignment rules of their Task Queue. Same goes for Child Workflows or Continue-As-New Workflows + // who inherit the parent/previous workflow's Build ID but not its Task Queue. In those cases, make + // sure to query reachability for the parent/previous workflow's Task Queue as well. + TaskReachability v1.BuildIdTaskReachability `protobuf:"varint,2,opt,name=task_reachability,json=taskReachability,proto3,enum=temporal.api.enums.v1.BuildIdTaskReachability" json:"task_reachability,omitempty"` +} + +func (x *TaskQueueVersionInfo) Reset() { + *x = TaskQueueVersionInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TaskQueueVersionInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskQueueVersionInfo) ProtoMessage() {} + +func (x *TaskQueueVersionInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TaskQueueVersionInfo.ProtoReflect.Descriptor instead. +func (*TaskQueueVersionInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *TaskQueueVersionInfo) GetTypesInfo() map[int32]*TaskQueueTypeInfo { + if x != nil { + return x.TypesInfo + } + return nil +} + +func (x *TaskQueueVersionInfo) GetTaskReachability() v1.BuildIdTaskReachability { + if x != nil { + return x.TaskReachability + } + return v1.BuildIdTaskReachability(0) +} + +type TaskQueueTypeInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Unversioned workers (with `useVersioning=false`) are reported in unversioned result even if they set a Build ID. + Pollers []*PollerInfo `protobuf:"bytes,1,rep,name=pollers,proto3" json:"pollers,omitempty"` + Stats *TaskQueueStats `protobuf:"bytes,2,opt,name=stats,proto3" json:"stats,omitempty"` +} + +func (x *TaskQueueTypeInfo) Reset() { + *x = TaskQueueTypeInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TaskQueueTypeInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskQueueTypeInfo) ProtoMessage() {} + +func (x *TaskQueueTypeInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TaskQueueTypeInfo.ProtoReflect.Descriptor instead. +func (*TaskQueueTypeInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *TaskQueueTypeInfo) GetPollers() []*PollerInfo { + if x != nil { + return x.Pollers + } + return nil +} + +func (x *TaskQueueTypeInfo) GetStats() *TaskQueueStats { + if x != nil { + return x.Stats + } + return nil +} + +// For workflow task queues, we only report the normal queue stats, not sticky queues. This means the stats +// reported here do not count all workflow tasks. However, because the tasks queued in sticky queues only remain +// valid for a few seconds, the inaccuracy becomes less significant as the backlog age grows. +type TaskQueueStats struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The approximate number of tasks backlogged in this task queue. May count expired tasks but eventually converges + // to the right value. + ApproximateBacklogCount int64 `protobuf:"varint,1,opt,name=approximate_backlog_count,json=approximateBacklogCount,proto3" json:"approximate_backlog_count,omitempty"` + // Approximate age of the oldest task in the backlog based on the create timestamp of the task at the head of the queue. + ApproximateBacklogAge *durationpb.Duration `protobuf:"bytes,2,opt,name=approximate_backlog_age,json=approximateBacklogAge,proto3" json:"approximate_backlog_age,omitempty"` + // Approximate tasks per second added to the task queue based on activity within a fixed window. This includes both backlogged and + // sync-matched tasks. + TasksAddRate float32 `protobuf:"fixed32,3,opt,name=tasks_add_rate,json=tasksAddRate,proto3" json:"tasks_add_rate,omitempty"` + // Approximate tasks per second dispatched to workers based on activity within a fixed window. This includes both backlogged and + // sync-matched tasks. + TasksDispatchRate float32 `protobuf:"fixed32,4,opt,name=tasks_dispatch_rate,json=tasksDispatchRate,proto3" json:"tasks_dispatch_rate,omitempty"` +} + +func (x *TaskQueueStats) Reset() { + *x = TaskQueueStats{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TaskQueueStats) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskQueueStats) ProtoMessage() {} + +func (x *TaskQueueStats) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TaskQueueStats.ProtoReflect.Descriptor instead. +func (*TaskQueueStats) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{5} +} + +func (x *TaskQueueStats) GetApproximateBacklogCount() int64 { + if x != nil { + return x.ApproximateBacklogCount + } + return 0 +} + +func (x *TaskQueueStats) GetApproximateBacklogAge() *durationpb.Duration { + if x != nil { + return x.ApproximateBacklogAge + } + return nil +} + +func (x *TaskQueueStats) GetTasksAddRate() float32 { + if x != nil { + return x.TasksAddRate + } + return 0 +} + +func (x *TaskQueueStats) GetTasksDispatchRate() float32 { + if x != nil { + return x.TasksDispatchRate + } + return 0 +} + +// Deprecated. Use `InternalTaskQueueStatus`. This is kept until `DescribeTaskQueue` supports legacy behavior. +type TaskQueueStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + BacklogCountHint int64 `protobuf:"varint,1,opt,name=backlog_count_hint,json=backlogCountHint,proto3" json:"backlog_count_hint,omitempty"` + ReadLevel int64 `protobuf:"varint,2,opt,name=read_level,json=readLevel,proto3" json:"read_level,omitempty"` + AckLevel int64 `protobuf:"varint,3,opt,name=ack_level,json=ackLevel,proto3" json:"ack_level,omitempty"` + RatePerSecond float64 `protobuf:"fixed64,4,opt,name=rate_per_second,json=ratePerSecond,proto3" json:"rate_per_second,omitempty"` + TaskIdBlock *TaskIdBlock `protobuf:"bytes,5,opt,name=task_id_block,json=taskIdBlock,proto3" json:"task_id_block,omitempty"` +} + +func (x *TaskQueueStatus) Reset() { + *x = TaskQueueStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TaskQueueStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskQueueStatus) ProtoMessage() {} + +func (x *TaskQueueStatus) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TaskQueueStatus.ProtoReflect.Descriptor instead. +func (*TaskQueueStatus) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{6} +} + +func (x *TaskQueueStatus) GetBacklogCountHint() int64 { + if x != nil { + return x.BacklogCountHint + } + return 0 +} + +func (x *TaskQueueStatus) GetReadLevel() int64 { + if x != nil { + return x.ReadLevel + } + return 0 +} + +func (x *TaskQueueStatus) GetAckLevel() int64 { + if x != nil { + return x.AckLevel + } + return 0 +} + +func (x *TaskQueueStatus) GetRatePerSecond() float64 { + if x != nil { + return x.RatePerSecond + } + return 0 +} + +func (x *TaskQueueStatus) GetTaskIdBlock() *TaskIdBlock { + if x != nil { + return x.TaskIdBlock + } + return nil +} + +type TaskIdBlock struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + StartId int64 `protobuf:"varint,1,opt,name=start_id,json=startId,proto3" json:"start_id,omitempty"` + EndId int64 `protobuf:"varint,2,opt,name=end_id,json=endId,proto3" json:"end_id,omitempty"` +} + +func (x *TaskIdBlock) Reset() { + *x = TaskIdBlock{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TaskIdBlock) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskIdBlock) ProtoMessage() {} + +func (x *TaskIdBlock) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TaskIdBlock.ProtoReflect.Descriptor instead. +func (*TaskIdBlock) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{7} +} + +func (x *TaskIdBlock) GetStartId() int64 { + if x != nil { + return x.StartId + } + return 0 +} + +func (x *TaskIdBlock) GetEndId() int64 { + if x != nil { + return x.EndId + } + return 0 +} + +type TaskQueuePartitionMetadata struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + OwnerHostName string `protobuf:"bytes,2,opt,name=owner_host_name,json=ownerHostName,proto3" json:"owner_host_name,omitempty"` +} + +func (x *TaskQueuePartitionMetadata) Reset() { + *x = TaskQueuePartitionMetadata{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TaskQueuePartitionMetadata) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskQueuePartitionMetadata) ProtoMessage() {} + +func (x *TaskQueuePartitionMetadata) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TaskQueuePartitionMetadata.ProtoReflect.Descriptor instead. +func (*TaskQueuePartitionMetadata) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{8} +} + +func (x *TaskQueuePartitionMetadata) GetKey() string { + if x != nil { + return x.Key + } + return "" +} + +func (x *TaskQueuePartitionMetadata) GetOwnerHostName() string { + if x != nil { + return x.OwnerHostName + } + return "" +} + +type PollerInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + LastAccessTime *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=last_access_time,json=lastAccessTime,proto3" json:"last_access_time,omitempty"` + Identity string `protobuf:"bytes,2,opt,name=identity,proto3" json:"identity,omitempty"` + RatePerSecond float64 `protobuf:"fixed64,3,opt,name=rate_per_second,json=ratePerSecond,proto3" json:"rate_per_second,omitempty"` + // If a worker has opted into the worker versioning feature while polling, its capabilities will + // appear here. + WorkerVersionCapabilities *v11.WorkerVersionCapabilities `protobuf:"bytes,4,opt,name=worker_version_capabilities,json=workerVersionCapabilities,proto3" json:"worker_version_capabilities,omitempty"` +} + +func (x *PollerInfo) Reset() { + *x = PollerInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PollerInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PollerInfo) ProtoMessage() {} + +func (x *PollerInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PollerInfo.ProtoReflect.Descriptor instead. +func (*PollerInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{9} +} + +func (x *PollerInfo) GetLastAccessTime() *timestamppb.Timestamp { + if x != nil { + return x.LastAccessTime + } + return nil +} + +func (x *PollerInfo) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *PollerInfo) GetRatePerSecond() float64 { + if x != nil { + return x.RatePerSecond + } + return 0 +} + +func (x *PollerInfo) GetWorkerVersionCapabilities() *v11.WorkerVersionCapabilities { + if x != nil { + return x.WorkerVersionCapabilities + } + return nil +} + +type StickyExecutionAttributes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + WorkerTaskQueue *TaskQueue `protobuf:"bytes,1,opt,name=worker_task_queue,json=workerTaskQueue,proto3" json:"worker_task_queue,omitempty"` + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + ScheduleToStartTimeout *durationpb.Duration `protobuf:"bytes,2,opt,name=schedule_to_start_timeout,json=scheduleToStartTimeout,proto3" json:"schedule_to_start_timeout,omitempty"` +} + +func (x *StickyExecutionAttributes) Reset() { + *x = StickyExecutionAttributes{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StickyExecutionAttributes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StickyExecutionAttributes) ProtoMessage() {} + +func (x *StickyExecutionAttributes) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StickyExecutionAttributes.ProtoReflect.Descriptor instead. +func (*StickyExecutionAttributes) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{10} +} + +func (x *StickyExecutionAttributes) GetWorkerTaskQueue() *TaskQueue { + if x != nil { + return x.WorkerTaskQueue + } + return nil +} + +func (x *StickyExecutionAttributes) GetScheduleToStartTimeout() *durationpb.Duration { + if x != nil { + return x.ScheduleToStartTimeout + } + return nil +} + +// Used by the worker versioning APIs, represents an unordered set of one or more versions which are +// considered to be compatible with each other. Currently the versions are always worker build IDs. +type CompatibleVersionSet struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // All the compatible versions, unordered, except for the last element, which is considered the set "default". + BuildIds []string `protobuf:"bytes,1,rep,name=build_ids,json=buildIds,proto3" json:"build_ids,omitempty"` +} + +func (x *CompatibleVersionSet) Reset() { + *x = CompatibleVersionSet{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CompatibleVersionSet) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CompatibleVersionSet) ProtoMessage() {} + +func (x *CompatibleVersionSet) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CompatibleVersionSet.ProtoReflect.Descriptor instead. +func (*CompatibleVersionSet) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{11} +} + +func (x *CompatibleVersionSet) GetBuildIds() []string { + if x != nil { + return x.BuildIds + } + return nil +} + +// Reachability of tasks for a worker on a single task queue. +type TaskQueueReachability struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TaskQueue string `protobuf:"bytes,1,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + // Task reachability for a worker in a single task queue. + // See the TaskReachability docstring for information about each enum variant. + // If reachability is empty, this worker is considered unreachable in this task queue. + Reachability []v1.TaskReachability `protobuf:"varint,2,rep,packed,name=reachability,proto3,enum=temporal.api.enums.v1.TaskReachability" json:"reachability,omitempty"` +} + +func (x *TaskQueueReachability) Reset() { + *x = TaskQueueReachability{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TaskQueueReachability) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskQueueReachability) ProtoMessage() {} + +func (x *TaskQueueReachability) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TaskQueueReachability.ProtoReflect.Descriptor instead. +func (*TaskQueueReachability) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{12} +} + +func (x *TaskQueueReachability) GetTaskQueue() string { + if x != nil { + return x.TaskQueue + } + return "" +} + +func (x *TaskQueueReachability) GetReachability() []v1.TaskReachability { + if x != nil { + return x.Reachability + } + return nil +} + +// Reachability of tasks for a worker by build id, in one or more task queues. +type BuildIdReachability struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // A build id or empty if unversioned. + BuildId string `protobuf:"bytes,1,opt,name=build_id,json=buildId,proto3" json:"build_id,omitempty"` + // Reachability per task queue. + TaskQueueReachability []*TaskQueueReachability `protobuf:"bytes,2,rep,name=task_queue_reachability,json=taskQueueReachability,proto3" json:"task_queue_reachability,omitempty"` +} + +func (x *BuildIdReachability) Reset() { + *x = BuildIdReachability{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BuildIdReachability) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BuildIdReachability) ProtoMessage() {} + +func (x *BuildIdReachability) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BuildIdReachability.ProtoReflect.Descriptor instead. +func (*BuildIdReachability) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{13} +} + +func (x *BuildIdReachability) GetBuildId() string { + if x != nil { + return x.BuildId + } + return "" +} + +func (x *BuildIdReachability) GetTaskQueueReachability() []*TaskQueueReachability { + if x != nil { + return x.TaskQueueReachability + } + return nil +} + +type RampByPercentage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Acceptable range is [0,100). + RampPercentage float32 `protobuf:"fixed32,1,opt,name=ramp_percentage,json=rampPercentage,proto3" json:"ramp_percentage,omitempty"` +} + +func (x *RampByPercentage) Reset() { + *x = RampByPercentage{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RampByPercentage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RampByPercentage) ProtoMessage() {} + +func (x *RampByPercentage) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RampByPercentage.ProtoReflect.Descriptor instead. +func (*RampByPercentage) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{14} +} + +func (x *RampByPercentage) GetRampPercentage() float32 { + if x != nil { + return x.RampPercentage + } + return 0 +} + +// Assignment rules are applied to *new* Workflow and Activity executions at +// schedule time to assign them to a Build ID. +// +// Assignment rules will not be used in the following cases: +// - Child Workflows or Continue-As-New Executions who inherit their +// parent/previous Workflow's assigned Build ID (by setting the +// `inherit_build_id` flag - default behavior in SDKs when the same Task Queue +// is used.) +// - An Activity that inherits the assigned Build ID of its Workflow (by +// setting the `use_workflow_build_id` flag - default behavior in SDKs +// when the same Task Queue is used.) +// +// In absence of (applicable) redirect rules (`CompatibleBuildIdRedirectRule`s) +// the task will be dispatched to Workers of the Build ID determined by the +// assignment rules (or inherited). Otherwise, the final Build ID will be +// determined by the redirect rules. +// +// Once a Workflow completes its first Workflow Task in a particular Build ID it +// stays in that Build ID regardless of changes to assignment rules. Redirect +// rules can be used to move the workflow to another compatible Build ID. +// +// When using Worker Versioning on a Task Queue, in the steady state, +// there should typically be a single assignment rule to send all new executions +// to the latest Build ID. Existence of at least one such "unconditional" +// rule at all times is enforces by the system, unless the `force` flag is used +// by the user when replacing/deleting these rules (for exceptional cases). +// +// During a deployment, one or more additional rules can be added to assign a +// subset of the tasks to a new Build ID based on a "ramp percentage". +// +// When there are multiple assignment rules for a Task Queue, the rules are +// evaluated in order, starting from index 0. The first applicable rule will be +// applied and the rest will be ignored. +// +// In the event that no assignment rule is applicable on a task (or the Task +// Queue is simply not versioned), the tasks will be dispatched to an +// unversioned Worker. +type BuildIdAssignmentRule struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TargetBuildId string `protobuf:"bytes,1,opt,name=target_build_id,json=targetBuildId,proto3" json:"target_build_id,omitempty"` + // If a ramp is provided, this rule will be applied only to a sample of + // tasks according to the provided percentage. + // This option can be used only on "terminal" Build IDs (the ones not used + // as source in any redirect rules). + // + // Types that are assignable to Ramp: + // + // *BuildIdAssignmentRule_PercentageRamp + Ramp isBuildIdAssignmentRule_Ramp `protobuf_oneof:"ramp"` +} + +func (x *BuildIdAssignmentRule) Reset() { + *x = BuildIdAssignmentRule{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BuildIdAssignmentRule) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BuildIdAssignmentRule) ProtoMessage() {} + +func (x *BuildIdAssignmentRule) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[15] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BuildIdAssignmentRule.ProtoReflect.Descriptor instead. +func (*BuildIdAssignmentRule) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{15} +} + +func (x *BuildIdAssignmentRule) GetTargetBuildId() string { + if x != nil { + return x.TargetBuildId + } + return "" +} + +func (m *BuildIdAssignmentRule) GetRamp() isBuildIdAssignmentRule_Ramp { + if m != nil { + return m.Ramp + } + return nil +} + +func (x *BuildIdAssignmentRule) GetPercentageRamp() *RampByPercentage { + if x, ok := x.GetRamp().(*BuildIdAssignmentRule_PercentageRamp); ok { + return x.PercentageRamp + } + return nil +} + +type isBuildIdAssignmentRule_Ramp interface { + isBuildIdAssignmentRule_Ramp() +} + +type BuildIdAssignmentRule_PercentageRamp struct { + // This ramp is useful for gradual Blue/Green deployments (and similar) + // where you want to send a certain portion of the traffic to the target + // Build ID. + PercentageRamp *RampByPercentage `protobuf:"bytes,3,opt,name=percentage_ramp,json=percentageRamp,proto3,oneof"` +} + +func (*BuildIdAssignmentRule_PercentageRamp) isBuildIdAssignmentRule_Ramp() {} + +// These rules apply to tasks assigned to a particular Build ID +// (`source_build_id`) to redirect them to another *compatible* Build ID +// (`target_build_id`). +// +// It is user's responsibility to ensure that the target Build ID is compatible +// with the source Build ID (e.g. by using the Patching API). +// +// Most deployments are not expected to need these rules, however following +// situations can greatly benefit from redirects: +// - Need to move long-running Workflow Executions from an old Build ID to a +// newer one. +// - Need to hotfix some broken or stuck Workflow Executions. +// +// In steady state, redirect rules are beneficial when dealing with old +// Executions ran on now-decommissioned Build IDs: +// - To redirecting the Workflow Queries to the current (compatible) Build ID. +// - To be able to Reset an old Execution so it can run on the current +// (compatible) Build ID. +// +// Redirect rules can be chained. +type CompatibleBuildIdRedirectRule struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SourceBuildId string `protobuf:"bytes,1,opt,name=source_build_id,json=sourceBuildId,proto3" json:"source_build_id,omitempty"` + // Target Build ID must be compatible with the Source Build ID; that is it + // must be able to process event histories made by the Source Build ID by + // using [Patching](https://docs.temporal.io/workflows#patching) or other + // means. + TargetBuildId string `protobuf:"bytes,2,opt,name=target_build_id,json=targetBuildId,proto3" json:"target_build_id,omitempty"` +} + +func (x *CompatibleBuildIdRedirectRule) Reset() { + *x = CompatibleBuildIdRedirectRule{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CompatibleBuildIdRedirectRule) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CompatibleBuildIdRedirectRule) ProtoMessage() {} + +func (x *CompatibleBuildIdRedirectRule) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[16] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CompatibleBuildIdRedirectRule.ProtoReflect.Descriptor instead. +func (*CompatibleBuildIdRedirectRule) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{16} +} + +func (x *CompatibleBuildIdRedirectRule) GetSourceBuildId() string { + if x != nil { + return x.SourceBuildId + } + return "" +} + +func (x *CompatibleBuildIdRedirectRule) GetTargetBuildId() string { + if x != nil { + return x.TargetBuildId + } + return "" +} + +type TimestampedBuildIdAssignmentRule struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Rule *BuildIdAssignmentRule `protobuf:"bytes,1,opt,name=rule,proto3" json:"rule,omitempty"` + CreateTime *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` +} + +func (x *TimestampedBuildIdAssignmentRule) Reset() { + *x = TimestampedBuildIdAssignmentRule{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TimestampedBuildIdAssignmentRule) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimestampedBuildIdAssignmentRule) ProtoMessage() {} + +func (x *TimestampedBuildIdAssignmentRule) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[17] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TimestampedBuildIdAssignmentRule.ProtoReflect.Descriptor instead. +func (*TimestampedBuildIdAssignmentRule) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{17} +} + +func (x *TimestampedBuildIdAssignmentRule) GetRule() *BuildIdAssignmentRule { + if x != nil { + return x.Rule + } + return nil +} + +func (x *TimestampedBuildIdAssignmentRule) GetCreateTime() *timestamppb.Timestamp { + if x != nil { + return x.CreateTime + } + return nil +} + +type TimestampedCompatibleBuildIdRedirectRule struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Rule *CompatibleBuildIdRedirectRule `protobuf:"bytes,1,opt,name=rule,proto3" json:"rule,omitempty"` + CreateTime *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` +} + +func (x *TimestampedCompatibleBuildIdRedirectRule) Reset() { + *x = TimestampedCompatibleBuildIdRedirectRule{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TimestampedCompatibleBuildIdRedirectRule) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimestampedCompatibleBuildIdRedirectRule) ProtoMessage() {} + +func (x *TimestampedCompatibleBuildIdRedirectRule) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_taskqueue_v1_message_proto_msgTypes[18] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TimestampedCompatibleBuildIdRedirectRule.ProtoReflect.Descriptor instead. +func (*TimestampedCompatibleBuildIdRedirectRule) Descriptor() ([]byte, []int) { + return file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP(), []int{18} +} + +func (x *TimestampedCompatibleBuildIdRedirectRule) GetRule() *CompatibleBuildIdRedirectRule { + if x != nil { + return x.Rule + } + return nil +} + +func (x *TimestampedCompatibleBuildIdRedirectRule) GetCreateTime() *timestamppb.Timestamp { + if x != nil { + return x.CreateTime + } + return nil +} + +var File_temporal_api_taskqueue_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_taskqueue_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x27, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x74, 0x61, + 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x19, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, + 0x1a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x77, + 0x72, 0x61, 0x70, 0x70, 0x65, 0x72, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x26, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, + 0x76, 0x31, 0x2f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x86, 0x01, 0x0a, 0x09, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, + 0x65, 0x75, 0x65, 0x12, 0x16, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3c, 0x0a, 0x04, 0x6b, 0x69, 0x6e, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x54, + 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x4b, 0x69, 0x6e, 0x64, 0x52, 0x04, 0x6b, 0x69, 0x6e, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x6e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, + 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x66, 0x0a, 0x11, 0x54, 0x61, 0x73, 0x6b, 0x51, + 0x75, 0x65, 0x75, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x51, 0x0a, 0x14, 0x6d, + 0x61, 0x78, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x63, 0x6f, + 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x6f, 0x75, 0x62, 0x6c, 0x65, + 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x11, 0x6d, 0x61, 0x78, 0x54, 0x61, 0x73, 0x6b, 0x73, 0x50, 0x65, + 0x72, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x85, 0x01, 0x0a, 0x19, 0x54, + 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x65, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x09, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, + 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, + 0x64, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x24, 0x0a, 0x0b, 0x75, 0x6e, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x75, 0x6e, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x61, 0x6c, + 0x6c, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, + 0x6c, 0x6c, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xce, 0x02, 0x0a, 0x14, + 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, + 0x6e, 0x66, 0x6f, 0x12, 0x61, 0x0a, 0x0a, 0x74, 0x79, 0x70, 0x65, 0x73, 0x5f, 0x69, 0x6e, 0x66, 0x6f, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x09, 0x74, 0x79, 0x70, 0x65, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x5f, 0x0a, 0x11, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, + 0x69, 0x6c, 0x69, 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, + 0x31, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x61, 0x63, + 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x10, 0x74, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x61, + 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x72, 0x0a, 0x0e, + 0x54, 0x79, 0x70, 0x65, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x46, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, + 0x51, 0x75, 0x65, 0x75, 0x65, 0x54, 0x79, 0x70, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9d, 0x01, 0x0a, 0x11, 0x54, + 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x54, 0x79, 0x70, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, + 0x43, 0x0a, 0x07, 0x70, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x25, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, + 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6f, 0x6c, 0x6c, 0x65, + 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x70, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x43, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, + 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, + 0x65, 0x75, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x85, 0x02, 0x0a, 0x0e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x53, + 0x74, 0x61, 0x74, 0x73, 0x12, 0x3e, 0x0a, 0x19, 0x61, 0x70, 0x70, 0x72, 0x6f, 0x78, 0x69, 0x6d, 0x61, + 0x74, 0x65, 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x17, 0x61, 0x70, 0x70, 0x72, 0x6f, 0x78, 0x69, 0x6d, 0x61, + 0x74, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x55, 0x0a, 0x17, 0x61, 0x70, 0x70, 0x72, 0x6f, 0x78, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x5f, + 0x62, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x5f, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x15, 0x61, 0x70, 0x70, 0x72, 0x6f, + 0x78, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x41, 0x67, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x28, 0x0a, 0x0e, 0x74, 0x61, 0x73, 0x6b, 0x73, 0x5f, 0x61, 0x64, 0x64, 0x5f, + 0x72, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02, 0x52, 0x0c, 0x74, 0x61, 0x73, 0x6b, + 0x73, 0x41, 0x64, 0x64, 0x52, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x32, 0x0a, 0x13, 0x74, + 0x61, 0x73, 0x6b, 0x73, 0x5f, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x72, 0x61, 0x74, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x02, 0x52, 0x11, 0x74, 0x61, 0x73, 0x6b, 0x73, 0x44, 0x69, 0x73, + 0x70, 0x61, 0x74, 0x63, 0x68, 0x52, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x83, 0x02, 0x0a, + 0x0f, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, + 0x30, 0x0a, 0x12, 0x62, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, + 0x68, 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x62, 0x61, 0x63, 0x6b, 0x6c, + 0x6f, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x48, 0x69, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, + 0x0a, 0x0a, 0x72, 0x65, 0x61, 0x64, 0x5f, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x09, 0x72, 0x65, 0x61, 0x64, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x1f, 0x0a, 0x09, 0x61, 0x63, 0x6b, 0x5f, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x08, 0x61, 0x63, 0x6b, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x2a, 0x0a, 0x0f, 0x72, 0x61, 0x74, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, + 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0d, 0x72, 0x61, 0x74, 0x65, 0x50, 0x65, 0x72, 0x53, + 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4e, 0x0a, 0x0d, 0x74, 0x61, 0x73, 0x6b, + 0x5f, 0x69, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, + 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x64, + 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x0b, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x42, 0x6c, 0x6f, 0x63, + 0x6b, 0x42, 0x02, 0x68, 0x00, 0x22, 0x47, 0x0a, 0x0b, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x42, 0x6c, + 0x6f, 0x63, 0x6b, 0x12, 0x1d, 0x0a, 0x08, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x73, 0x74, 0x61, 0x72, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x19, 0x0a, 0x06, 0x65, 0x6e, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x05, 0x65, 0x6e, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x5e, 0x0a, 0x1a, 0x54, 0x61, 0x73, + 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6f, + 0x77, 0x6e, 0x65, 0x72, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x48, 0x6f, 0x73, 0x74, 0x4e, 0x61, 0x6d, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x99, 0x02, 0x0a, 0x0a, 0x50, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x49, + 0x6e, 0x66, 0x6f, 0x12, 0x48, 0x0a, 0x10, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, + 0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x41, 0x63, 0x63, 0x65, 0x73, + 0x73, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x72, 0x61, 0x74, 0x65, 0x5f, + 0x70, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x52, + 0x0d, 0x72, 0x61, 0x74, 0x65, 0x50, 0x65, 0x72, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x75, 0x0a, 0x1b, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x5f, 0x63, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, + 0x69, 0x74, 0x69, 0x65, 0x73, 0x52, 0x19, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x22, 0xcb, 0x01, 0x0a, 0x19, 0x53, 0x74, 0x69, 0x63, 0x6b, 0x79, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x54, + 0x0a, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, + 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x0f, 0x77, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x58, 0x0a, 0x19, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x73, + 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x16, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x54, 0x6f, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, + 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0x37, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, + 0x6c, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x74, 0x12, 0x1f, 0x0a, 0x09, 0x62, + 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8b, 0x01, 0x0a, 0x15, + 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, 0x0c, 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x27, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, + 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, + 0x79, 0x52, 0x0c, 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x02, + 0x68, 0x00, 0x22, 0xa2, 0x01, 0x0a, 0x13, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x52, 0x65, 0x61, + 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x12, 0x1d, 0x0a, 0x08, 0x62, 0x75, 0x69, 0x6c, + 0x64, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x6c, 0x0a, 0x17, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, + 0x65, 0x75, 0x65, 0x5f, 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, + 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x15, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, + 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x3f, + 0x0a, 0x10, 0x52, 0x61, 0x6d, 0x70, 0x42, 0x79, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, + 0x65, 0x12, 0x2b, 0x0a, 0x0f, 0x72, 0x61, 0x6d, 0x70, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, + 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x02, 0x52, 0x0e, 0x72, 0x61, 0x6d, 0x70, 0x50, 0x65, + 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xa7, 0x01, 0x0a, 0x15, + 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, + 0x52, 0x75, 0x6c, 0x65, 0x12, 0x2a, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x62, + 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x74, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5a, + 0x0a, 0x0f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x5f, 0x72, 0x61, 0x6d, 0x70, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x52, 0x61, 0x6d, 0x70, 0x42, 0x79, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, + 0x48, 0x00, 0x52, 0x0e, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x52, 0x61, 0x6d, + 0x70, 0x42, 0x02, 0x68, 0x00, 0x42, 0x06, 0x0a, 0x04, 0x72, 0x61, 0x6d, 0x70, 0x22, 0x77, 0x0a, 0x1d, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, + 0x64, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x2a, 0x0a, 0x0f, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x74, + 0x61, 0x72, 0x67, 0x65, 0x74, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, + 0xad, 0x01, 0x0a, 0x20, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x65, 0x64, 0x42, 0x75, + 0x69, 0x6c, 0x64, 0x49, 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, + 0x6c, 0x65, 0x12, 0x48, 0x0a, 0x04, 0x72, 0x75, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x30, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, + 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, + 0x49, 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x52, + 0x04, 0x72, 0x75, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, + 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xbd, 0x01, 0x0a, 0x28, 0x54, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x65, 0x64, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x42, + 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, + 0x65, 0x12, 0x50, 0x0a, 0x04, 0x72, 0x75, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, + 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, + 0x62, 0x6c, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, + 0x74, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x04, 0x72, 0x75, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, + 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0a, 0x63, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x42, 0x98, 0x01, 0x0a, + 0x1c, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x29, 0x67, 0x6f, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x74, 0x61, + 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2f, 0x76, 0x31, 0x3b, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, + 0x65, 0x75, 0x65, 0xaa, 0x02, 0x1b, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, + 0x41, 0x70, 0x69, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x56, 0x31, + 0xea, 0x02, 0x1e, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, + 0x69, 0x3a, 0x3a, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_taskqueue_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_taskqueue_v1_message_proto_rawDescData = file_temporal_api_taskqueue_v1_message_proto_rawDesc +) + +func file_temporal_api_taskqueue_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_taskqueue_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_taskqueue_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_taskqueue_v1_message_proto_rawDescData) + }) + return file_temporal_api_taskqueue_v1_message_proto_rawDescData +} + +var file_temporal_api_taskqueue_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 20) +var file_temporal_api_taskqueue_v1_message_proto_goTypes = []any{ + (*TaskQueue)(nil), // 0: temporal.api.taskqueue.v1.TaskQueue + (*TaskQueueMetadata)(nil), // 1: temporal.api.taskqueue.v1.TaskQueueMetadata + (*TaskQueueVersionSelection)(nil), // 2: temporal.api.taskqueue.v1.TaskQueueVersionSelection + (*TaskQueueVersionInfo)(nil), // 3: temporal.api.taskqueue.v1.TaskQueueVersionInfo + (*TaskQueueTypeInfo)(nil), // 4: temporal.api.taskqueue.v1.TaskQueueTypeInfo + (*TaskQueueStats)(nil), // 5: temporal.api.taskqueue.v1.TaskQueueStats + (*TaskQueueStatus)(nil), // 6: temporal.api.taskqueue.v1.TaskQueueStatus + (*TaskIdBlock)(nil), // 7: temporal.api.taskqueue.v1.TaskIdBlock + (*TaskQueuePartitionMetadata)(nil), // 8: temporal.api.taskqueue.v1.TaskQueuePartitionMetadata + (*PollerInfo)(nil), // 9: temporal.api.taskqueue.v1.PollerInfo + (*StickyExecutionAttributes)(nil), // 10: temporal.api.taskqueue.v1.StickyExecutionAttributes + (*CompatibleVersionSet)(nil), // 11: temporal.api.taskqueue.v1.CompatibleVersionSet + (*TaskQueueReachability)(nil), // 12: temporal.api.taskqueue.v1.TaskQueueReachability + (*BuildIdReachability)(nil), // 13: temporal.api.taskqueue.v1.BuildIdReachability + (*RampByPercentage)(nil), // 14: temporal.api.taskqueue.v1.RampByPercentage + (*BuildIdAssignmentRule)(nil), // 15: temporal.api.taskqueue.v1.BuildIdAssignmentRule + (*CompatibleBuildIdRedirectRule)(nil), // 16: temporal.api.taskqueue.v1.CompatibleBuildIdRedirectRule + (*TimestampedBuildIdAssignmentRule)(nil), // 17: temporal.api.taskqueue.v1.TimestampedBuildIdAssignmentRule + (*TimestampedCompatibleBuildIdRedirectRule)(nil), // 18: temporal.api.taskqueue.v1.TimestampedCompatibleBuildIdRedirectRule + nil, // 19: temporal.api.taskqueue.v1.TaskQueueVersionInfo.TypesInfoEntry + (v1.TaskQueueKind)(0), // 20: temporal.api.enums.v1.TaskQueueKind + (*wrapperspb.DoubleValue)(nil), // 21: google.protobuf.DoubleValue + (v1.BuildIdTaskReachability)(0), // 22: temporal.api.enums.v1.BuildIdTaskReachability + (*durationpb.Duration)(nil), // 23: google.protobuf.Duration + (*timestamppb.Timestamp)(nil), // 24: google.protobuf.Timestamp + (*v11.WorkerVersionCapabilities)(nil), // 25: temporal.api.common.v1.WorkerVersionCapabilities + (v1.TaskReachability)(0), // 26: temporal.api.enums.v1.TaskReachability +} +var file_temporal_api_taskqueue_v1_message_proto_depIdxs = []int32{ + 20, // 0: temporal.api.taskqueue.v1.TaskQueue.kind:type_name -> temporal.api.enums.v1.TaskQueueKind + 21, // 1: temporal.api.taskqueue.v1.TaskQueueMetadata.max_tasks_per_second:type_name -> google.protobuf.DoubleValue + 19, // 2: temporal.api.taskqueue.v1.TaskQueueVersionInfo.types_info:type_name -> temporal.api.taskqueue.v1.TaskQueueVersionInfo.TypesInfoEntry + 22, // 3: temporal.api.taskqueue.v1.TaskQueueVersionInfo.task_reachability:type_name -> temporal.api.enums.v1.BuildIdTaskReachability + 9, // 4: temporal.api.taskqueue.v1.TaskQueueTypeInfo.pollers:type_name -> temporal.api.taskqueue.v1.PollerInfo + 5, // 5: temporal.api.taskqueue.v1.TaskQueueTypeInfo.stats:type_name -> temporal.api.taskqueue.v1.TaskQueueStats + 23, // 6: temporal.api.taskqueue.v1.TaskQueueStats.approximate_backlog_age:type_name -> google.protobuf.Duration + 7, // 7: temporal.api.taskqueue.v1.TaskQueueStatus.task_id_block:type_name -> temporal.api.taskqueue.v1.TaskIdBlock + 24, // 8: temporal.api.taskqueue.v1.PollerInfo.last_access_time:type_name -> google.protobuf.Timestamp + 25, // 9: temporal.api.taskqueue.v1.PollerInfo.worker_version_capabilities:type_name -> temporal.api.common.v1.WorkerVersionCapabilities + 0, // 10: temporal.api.taskqueue.v1.StickyExecutionAttributes.worker_task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 23, // 11: temporal.api.taskqueue.v1.StickyExecutionAttributes.schedule_to_start_timeout:type_name -> google.protobuf.Duration + 26, // 12: temporal.api.taskqueue.v1.TaskQueueReachability.reachability:type_name -> temporal.api.enums.v1.TaskReachability + 12, // 13: temporal.api.taskqueue.v1.BuildIdReachability.task_queue_reachability:type_name -> temporal.api.taskqueue.v1.TaskQueueReachability + 14, // 14: temporal.api.taskqueue.v1.BuildIdAssignmentRule.percentage_ramp:type_name -> temporal.api.taskqueue.v1.RampByPercentage + 15, // 15: temporal.api.taskqueue.v1.TimestampedBuildIdAssignmentRule.rule:type_name -> temporal.api.taskqueue.v1.BuildIdAssignmentRule + 24, // 16: temporal.api.taskqueue.v1.TimestampedBuildIdAssignmentRule.create_time:type_name -> google.protobuf.Timestamp + 16, // 17: temporal.api.taskqueue.v1.TimestampedCompatibleBuildIdRedirectRule.rule:type_name -> temporal.api.taskqueue.v1.CompatibleBuildIdRedirectRule + 24, // 18: temporal.api.taskqueue.v1.TimestampedCompatibleBuildIdRedirectRule.create_time:type_name -> google.protobuf.Timestamp + 4, // 19: temporal.api.taskqueue.v1.TaskQueueVersionInfo.TypesInfoEntry.value:type_name -> temporal.api.taskqueue.v1.TaskQueueTypeInfo + 20, // [20:20] is the sub-list for method output_type + 20, // [20:20] is the sub-list for method input_type + 20, // [20:20] is the sub-list for extension type_name + 20, // [20:20] is the sub-list for extension extendee + 0, // [0:20] is the sub-list for field type_name +} + +func init() { file_temporal_api_taskqueue_v1_message_proto_init() } +func file_temporal_api_taskqueue_v1_message_proto_init() { + if File_temporal_api_taskqueue_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_taskqueue_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*TaskQueue); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*TaskQueueMetadata); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*TaskQueueVersionSelection); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*TaskQueueVersionInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*TaskQueueTypeInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*TaskQueueStats); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*TaskQueueStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*TaskIdBlock); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*TaskQueuePartitionMetadata); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*PollerInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*StickyExecutionAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[11].Exporter = func(v any, i int) any { + switch v := v.(*CompatibleVersionSet); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[12].Exporter = func(v any, i int) any { + switch v := v.(*TaskQueueReachability); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[13].Exporter = func(v any, i int) any { + switch v := v.(*BuildIdReachability); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[14].Exporter = func(v any, i int) any { + switch v := v.(*RampByPercentage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[15].Exporter = func(v any, i int) any { + switch v := v.(*BuildIdAssignmentRule); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[16].Exporter = func(v any, i int) any { + switch v := v.(*CompatibleBuildIdRedirectRule); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[17].Exporter = func(v any, i int) any { + switch v := v.(*TimestampedBuildIdAssignmentRule); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[18].Exporter = func(v any, i int) any { + switch v := v.(*TimestampedCompatibleBuildIdRedirectRule); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_temporal_api_taskqueue_v1_message_proto_msgTypes[15].OneofWrappers = []any{ + (*BuildIdAssignmentRule_PercentageRamp)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_taskqueue_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 20, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_taskqueue_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_taskqueue_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_taskqueue_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_taskqueue_v1_message_proto = out.File + file_temporal_api_taskqueue_v1_message_proto_rawDesc = nil + file_temporal_api_taskqueue_v1_message_proto_goTypes = nil + file_temporal_api_taskqueue_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/temporalproto/deep_equal.go b/vendor/go.temporal.io/api/temporalproto/deep_equal.go new file mode 100644 index 00000000000..cb16a98f10e --- /dev/null +++ b/vendor/go.temporal.io/api/temporalproto/deep_equal.go @@ -0,0 +1,214 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Copyright 2009 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Deep equality test via reflection on only public methods and members. +// This is deeply hacky as we're not inside of the reflect package; this will not +// be as performant as reflect.DeepEqual but will handle proto objects at any point +// in an object hierarchy +package temporalproto + +import ( + "reflect" + "regexp" + "unsafe" + + "google.golang.org/protobuf/proto" +) + +// During deepValueEqual, must keep track of checks that are +// in progress. The comparison algorithm assumes that all +// checks in progress are true when it reencounters them. +// Visited comparisons are stored in a map indexed by visit. +type visit struct { + a1 unsafe.Pointer + a2 unsafe.Pointer + typ reflect.Type +} + +var publicMethodRgx = regexp.MustCompile("^[A-Z]") + +func pointerTo(v reflect.Value) unsafe.Pointer { + if v.CanAddr() { + return v.Addr().UnsafePointer() + } + return v.UnsafePointer() +} + +// Tests for deep equality using reflected types. The map argument tracks +// comparisons that have already been seen, which allows short circuiting on +// recursive types. +func deepValueEqual(v1, v2 reflect.Value, visited map[visit]bool) bool { + if !v1.IsValid() || !v2.IsValid() { + return v1.IsValid() == v2.IsValid() + } + if v1.Type() != v2.Type() { + return false + } + + // We want to avoid putting more in the visited map than we need to. + // For any possible reference cycle that might be encountered, + // hard(v1, v2) needs to return true for at least one of the types in the cycle, + // and it's safe and valid to get Value's internal pointer. + hard := func(v1, v2 reflect.Value) bool { + switch v1.Kind() { + case reflect.Pointer, reflect.Map, reflect.Slice, reflect.Interface: + // Nil pointers cannot be cyclic. Avoid putting them in the visited map. + return !v1.IsNil() && !v2.IsNil() + } + return false + } + + if hard(v1, v2) { + addr1 := pointerTo(v1) + addr2 := pointerTo(v2) + if uintptr(addr1) > uintptr(addr2) { + // Canonicalize order to reduce number of entries in visited. + // Assumes non-moving garbage collector. + addr1, addr2 = addr2, addr1 + } + + // Short circuit if references are already seen. + typ := v1.Type() + v := visit{addr1, addr2, typ} + if visited[v] { + return true + } + + // Remember for later. + visited[v] = true + } + + switch v1.Kind() { + case reflect.Array: + for i := 0; i < v1.Len(); i++ { + if !deepValueEqual(v1.Index(i), v2.Index(i), visited) { + return false + } + } + return true + case reflect.Slice: + if v1.IsNil() != v2.IsNil() { + return false + } + if v1.Len() != v2.Len() { + return false + } + if v1.UnsafePointer() == v2.UnsafePointer() { + return true + } + for i := 0; i < v1.Len(); i++ { + if !deepValueEqual(v1.Index(i), v2.Index(i), visited) { + return false + } + } + return true + case reflect.Interface: + if v1.IsNil() || v2.IsNil() { + return v1.IsNil() == v2.IsNil() + } + return deepValueEqual(v1.Elem(), v2.Elem(), visited) + case reflect.Pointer: + if v1.UnsafePointer() == v2.UnsafePointer() { + return true + } + + v1v, ok := v1.Interface().(proto.Message) + v2v, ok2 := v2.Interface().(proto.Message) + if ok && ok2 { + return proto.Equal(v1v, v2v) + } + + return deepValueEqual(v1.Elem(), v2.Elem(), visited) + case reflect.Struct: + for i, n := 0, v1.NumField(); i < n; i++ { + if !publicMethodRgx.MatchString(v1.Field(i).String()) { + continue + } + if !deepValueEqual(v1.Field(i), v2.Field(i), visited) { + return false + } + } + return true + case reflect.Map: + if v1.IsNil() != v2.IsNil() { + return false + } + if v1.Len() != v2.Len() { + return false + } + if v1.UnsafePointer() == v2.UnsafePointer() { + return true + } + for _, k := range v1.MapKeys() { + val1 := v1.MapIndex(k) + val2 := v2.MapIndex(k) + if !val1.IsValid() || !val2.IsValid() || !deepValueEqual(val1, val2, visited) { + return false + } + } + return true + case reflect.Func: + if v1.IsNil() && v2.IsNil() { + return true + } + // Can't do better than this: + return false + case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64: + return v1.Int() == v2.Int() + case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uintptr: + return v1.Uint() == v2.Uint() + case reflect.String: + return v1.String() == v2.String() + case reflect.Bool: + return v1.Bool() == v2.Bool() + case reflect.Float32, reflect.Float64: + return v1.Float() == v2.Float() + case reflect.Complex64, reflect.Complex128: + return v1.Complex() == v2.Complex() + default: + // Normal equality suffices + return v1.Elem().Interface() == v2.Elem().Interface() + } +} + +// DeepEqual behaves as reflect.DeepEqual except: +// 1. Proto structs will be compared using proto.Equal when encountered +// 2. Only public member variables will be compared +// +// DeepEqual should _only_ be used when proto.Equal or reflect.DeepEqual +// aren't useable, such as when comparing normal Go structs that have +// proto structs as members +func DeepEqual(x, y any) bool { + if x == nil || y == nil { + return x == y + } + v1 := reflect.ValueOf(x) + v2 := reflect.ValueOf(y) + if v1.Type() != v2.Type() { + return false + } + return deepValueEqual(v1, v2, make(map[visit]bool)) +} diff --git a/vendor/go.temporal.io/api/temporalproto/json_marshal.go b/vendor/go.temporal.io/api/temporalproto/json_marshal.go new file mode 100644 index 00000000000..e8cf5926ed2 --- /dev/null +++ b/vendor/go.temporal.io/api/temporalproto/json_marshal.go @@ -0,0 +1,41 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +package temporalproto + +import ( + "google.golang.org/protobuf/proto" + + "go.temporal.io/api/internal/protojson" +) + +// CustomJSONMarshalOptions is a configurable JSON format marshaler that supports +// both TYPE_PREFIXED_SCREAMING_SNAKE enums as well as camelCase enums. +type CustomJSONMarshalOptions struct { + // Metadata is used for storing request metadata, such as whether shorthand + // payloads are disabled + Metadata map[string]interface{} + + // Indent specifies the set of indentation characters to use in a multiline + // formatted output such that every entry is preceded by Indent and + // terminated by a newline. If non-empty, then Multiline is treated as true. + // Indent can only be composed of space or tab characters. + Indent string +} + +// Marshal marshals the given [proto.Message] in the JSON format using options in +// MarshalOptions. Do not depend on the output being stable. It may change over +// time across different versions of the program. +func (o CustomJSONMarshalOptions) Marshal(m proto.Message) ([]byte, error) { + return protojson.MarshalOptions{ + Indent: o.Indent, + Metadata: o.Metadata, + }.Marshal(m) +} diff --git a/vendor/go.temporal.io/api/temporalproto/json_unmarshal.go b/vendor/go.temporal.io/api/temporalproto/json_unmarshal.go new file mode 100644 index 00000000000..e8185f94593 --- /dev/null +++ b/vendor/go.temporal.io/api/temporalproto/json_unmarshal.go @@ -0,0 +1,42 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +package temporalproto + +import ( + "go.temporal.io/api/internal/protojson" + "google.golang.org/protobuf/proto" +) + +// CustomJSONUnmarshalOptions is a configurable JSON format marshaler that supports +// both TYPE_PREFIXED_SCREAMING_SNAKE enums as well as camelCase enums. +type CustomJSONUnmarshalOptions struct { + // Metadata is used for storing request metadata, such as whether shorthand + // payloads are disabled + Metadata map[string]interface{} + + // If DiscardUnknown is set, unknown fields and enum name values are ignored. + DiscardUnknown bool +} + +// Unmarshal reads the given []byte and populates the given [proto.Message] +// using options in the UnmarshalOptions object. +// It will clear the message first before setting the fields. +// If it returns an error, the given message may be partially set. +// The provided message must be mutable (e.g., a non-nil pointer to a message). +// This is different from the official protojson unmarshaling code in that it +// supports unmarshaling our shorthand payload format as well as both camelCase +// and SCREAMING_SNAKE_CASE JSON enums +func (o CustomJSONUnmarshalOptions) Unmarshal(b []byte, m proto.Message) error { + return protojson.UnmarshalOptions{ + Metadata: o.Metadata, + DiscardUnknown: o.DiscardUnknown, + }.Unmarshal(b, m) +} diff --git a/vendor/go.temporal.io/api/update/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/update/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..7f3884f004f --- /dev/null +++ b/vendor/go.temporal.io/api/update/v1/message.go-helpers.pb.go @@ -0,0 +1,361 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package update + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type WaitPolicy to the protobuf v3 wire format +func (val *WaitPolicy) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WaitPolicy from the protobuf v3 wire format +func (val *WaitPolicy) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WaitPolicy) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WaitPolicy values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WaitPolicy) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WaitPolicy + switch t := that.(type) { + case *WaitPolicy: + that1 = t + case WaitPolicy: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateRef to the protobuf v3 wire format +func (val *UpdateRef) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateRef from the protobuf v3 wire format +func (val *UpdateRef) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateRef) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateRef values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateRef) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateRef + switch t := that.(type) { + case *UpdateRef: + that1 = t + case UpdateRef: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Outcome to the protobuf v3 wire format +func (val *Outcome) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Outcome from the protobuf v3 wire format +func (val *Outcome) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Outcome) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Outcome values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Outcome) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Outcome + switch t := that.(type) { + case *Outcome: + that1 = t + case Outcome: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Meta to the protobuf v3 wire format +func (val *Meta) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Meta from the protobuf v3 wire format +func (val *Meta) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Meta) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Meta values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Meta) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Meta + switch t := that.(type) { + case *Meta: + that1 = t + case Meta: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Input to the protobuf v3 wire format +func (val *Input) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Input from the protobuf v3 wire format +func (val *Input) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Input) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Input values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Input) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Input + switch t := that.(type) { + case *Input: + that1 = t + case Input: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Request to the protobuf v3 wire format +func (val *Request) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Request from the protobuf v3 wire format +func (val *Request) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Request) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Request values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Request) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Request + switch t := that.(type) { + case *Request: + that1 = t + case Request: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Rejection to the protobuf v3 wire format +func (val *Rejection) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Rejection from the protobuf v3 wire format +func (val *Rejection) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Rejection) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Rejection values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Rejection) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Rejection + switch t := that.(type) { + case *Rejection: + that1 = t + case Rejection: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Acceptance to the protobuf v3 wire format +func (val *Acceptance) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Acceptance from the protobuf v3 wire format +func (val *Acceptance) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Acceptance) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Acceptance values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Acceptance) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Acceptance + switch t := that.(type) { + case *Acceptance: + that1 = t + case Acceptance: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Response to the protobuf v3 wire format +func (val *Response) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Response from the protobuf v3 wire format +func (val *Response) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Response) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Response values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Response) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Response + switch t := that.(type) { + case *Response: + that1 = t + case Response: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/update/v1/message.pb.go b/vendor/go.temporal.io/api/update/v1/message.pb.go new file mode 100644 index 00000000000..f5655baf716 --- /dev/null +++ b/vendor/go.temporal.io/api/update/v1/message.pb.go @@ -0,0 +1,913 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/update/v1/message.proto + +package update + +import ( + reflect "reflect" + sync "sync" + + v11 "go.temporal.io/api/common/v1" + v1 "go.temporal.io/api/enums/v1" + v12 "go.temporal.io/api/failure/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// Specifies to the gRPC server how long the client wants the an update-related +// RPC call to wait before returning control to the caller. +type WaitPolicy struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Indicates the update lifecycle stage that the gRPC call should wait for + // before returning. + LifecycleStage v1.UpdateWorkflowExecutionLifecycleStage `protobuf:"varint,1,opt,name=lifecycle_stage,json=lifecycleStage,proto3,enum=temporal.api.enums.v1.UpdateWorkflowExecutionLifecycleStage" json:"lifecycle_stage,omitempty"` +} + +func (x *WaitPolicy) Reset() { + *x = WaitPolicy{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WaitPolicy) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WaitPolicy) ProtoMessage() {} + +func (x *WaitPolicy) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WaitPolicy.ProtoReflect.Descriptor instead. +func (*WaitPolicy) Descriptor() ([]byte, []int) { + return file_temporal_api_update_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *WaitPolicy) GetLifecycleStage() v1.UpdateWorkflowExecutionLifecycleStage { + if x != nil { + return x.LifecycleStage + } + return v1.UpdateWorkflowExecutionLifecycleStage(0) +} + +// The data needed by a client to refer to a previously invoked workflow +// execution update process. +type UpdateRef struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + WorkflowExecution *v11.WorkflowExecution `protobuf:"bytes,1,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + UpdateId string `protobuf:"bytes,2,opt,name=update_id,json=updateId,proto3" json:"update_id,omitempty"` +} + +func (x *UpdateRef) Reset() { + *x = UpdateRef{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateRef) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateRef) ProtoMessage() {} + +func (x *UpdateRef) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateRef.ProtoReflect.Descriptor instead. +func (*UpdateRef) Descriptor() ([]byte, []int) { + return file_temporal_api_update_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *UpdateRef) GetWorkflowExecution() *v11.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *UpdateRef) GetUpdateId() string { + if x != nil { + return x.UpdateId + } + return "" +} + +// The outcome of a workflow update - success or failure. +type Outcome struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Value: + // + // *Outcome_Success + // *Outcome_Failure + Value isOutcome_Value `protobuf_oneof:"value"` +} + +func (x *Outcome) Reset() { + *x = Outcome{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Outcome) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Outcome) ProtoMessage() {} + +func (x *Outcome) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Outcome.ProtoReflect.Descriptor instead. +func (*Outcome) Descriptor() ([]byte, []int) { + return file_temporal_api_update_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (m *Outcome) GetValue() isOutcome_Value { + if m != nil { + return m.Value + } + return nil +} + +func (x *Outcome) GetSuccess() *v11.Payloads { + if x, ok := x.GetValue().(*Outcome_Success); ok { + return x.Success + } + return nil +} + +func (x *Outcome) GetFailure() *v12.Failure { + if x, ok := x.GetValue().(*Outcome_Failure); ok { + return x.Failure + } + return nil +} + +type isOutcome_Value interface { + isOutcome_Value() +} + +type Outcome_Success struct { + Success *v11.Payloads `protobuf:"bytes,1,opt,name=success,proto3,oneof"` +} + +type Outcome_Failure struct { + Failure *v12.Failure `protobuf:"bytes,2,opt,name=failure,proto3,oneof"` +} + +func (*Outcome_Success) isOutcome_Value() {} + +func (*Outcome_Failure) isOutcome_Value() {} + +// Metadata about a workflow execution update. +type Meta struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // An ID with workflow-scoped uniqueness for this update + UpdateId string `protobuf:"bytes,1,opt,name=update_id,json=updateId,proto3" json:"update_id,omitempty"` + // A string identifying the agent that requested this update. + Identity string `protobuf:"bytes,2,opt,name=identity,proto3" json:"identity,omitempty"` +} + +func (x *Meta) Reset() { + *x = Meta{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Meta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Meta) ProtoMessage() {} + +func (x *Meta) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Meta.ProtoReflect.Descriptor instead. +func (*Meta) Descriptor() ([]byte, []int) { + return file_temporal_api_update_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *Meta) GetUpdateId() string { + if x != nil { + return x.UpdateId + } + return "" +} + +func (x *Meta) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +type Input struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Headers that are passed with the update from the requesting entity. + // These can include things like auth or tracing tokens. + Header *v11.Header `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"` + // The name of the input handler to invoke on the target workflow + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + // The arguments to pass to the named handler. + Args *v11.Payloads `protobuf:"bytes,3,opt,name=args,proto3" json:"args,omitempty"` +} + +func (x *Input) Reset() { + *x = Input{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Input) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Input) ProtoMessage() {} + +func (x *Input) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Input.ProtoReflect.Descriptor instead. +func (*Input) Descriptor() ([]byte, []int) { + return file_temporal_api_update_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *Input) GetHeader() *v11.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *Input) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Input) GetArgs() *v11.Payloads { + if x != nil { + return x.Args + } + return nil +} + +// The client request that triggers a workflow execution update +type Request struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Meta *Meta `protobuf:"bytes,1,opt,name=meta,proto3" json:"meta,omitempty"` + Input *Input `protobuf:"bytes,2,opt,name=input,proto3" json:"input,omitempty"` +} + +func (x *Request) Reset() { + *x = Request{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Request) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Request) ProtoMessage() {} + +func (x *Request) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Request.ProtoReflect.Descriptor instead. +func (*Request) Descriptor() ([]byte, []int) { + return file_temporal_api_update_v1_message_proto_rawDescGZIP(), []int{5} +} + +func (x *Request) GetMeta() *Meta { + if x != nil { + return x.Meta + } + return nil +} + +func (x *Request) GetInput() *Input { + if x != nil { + return x.Input + } + return nil +} + +// An update protocol message indicating that a workflow execution update has +// been rejected. +type Rejection struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + RejectedRequestMessageId string `protobuf:"bytes,1,opt,name=rejected_request_message_id,json=rejectedRequestMessageId,proto3" json:"rejected_request_message_id,omitempty"` + RejectedRequestSequencingEventId int64 `protobuf:"varint,2,opt,name=rejected_request_sequencing_event_id,json=rejectedRequestSequencingEventId,proto3" json:"rejected_request_sequencing_event_id,omitempty"` + RejectedRequest *Request `protobuf:"bytes,3,opt,name=rejected_request,json=rejectedRequest,proto3" json:"rejected_request,omitempty"` + Failure *v12.Failure `protobuf:"bytes,4,opt,name=failure,proto3" json:"failure,omitempty"` +} + +func (x *Rejection) Reset() { + *x = Rejection{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Rejection) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Rejection) ProtoMessage() {} + +func (x *Rejection) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Rejection.ProtoReflect.Descriptor instead. +func (*Rejection) Descriptor() ([]byte, []int) { + return file_temporal_api_update_v1_message_proto_rawDescGZIP(), []int{6} +} + +func (x *Rejection) GetRejectedRequestMessageId() string { + if x != nil { + return x.RejectedRequestMessageId + } + return "" +} + +func (x *Rejection) GetRejectedRequestSequencingEventId() int64 { + if x != nil { + return x.RejectedRequestSequencingEventId + } + return 0 +} + +func (x *Rejection) GetRejectedRequest() *Request { + if x != nil { + return x.RejectedRequest + } + return nil +} + +func (x *Rejection) GetFailure() *v12.Failure { + if x != nil { + return x.Failure + } + return nil +} + +// An update protocol message indicating that a workflow execution update has +// been accepted (i.e. passed the worker-side validation phase). +type Acceptance struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + AcceptedRequestMessageId string `protobuf:"bytes,1,opt,name=accepted_request_message_id,json=acceptedRequestMessageId,proto3" json:"accepted_request_message_id,omitempty"` + AcceptedRequestSequencingEventId int64 `protobuf:"varint,2,opt,name=accepted_request_sequencing_event_id,json=acceptedRequestSequencingEventId,proto3" json:"accepted_request_sequencing_event_id,omitempty"` + AcceptedRequest *Request `protobuf:"bytes,3,opt,name=accepted_request,json=acceptedRequest,proto3" json:"accepted_request,omitempty"` +} + +func (x *Acceptance) Reset() { + *x = Acceptance{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Acceptance) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Acceptance) ProtoMessage() {} + +func (x *Acceptance) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Acceptance.ProtoReflect.Descriptor instead. +func (*Acceptance) Descriptor() ([]byte, []int) { + return file_temporal_api_update_v1_message_proto_rawDescGZIP(), []int{7} +} + +func (x *Acceptance) GetAcceptedRequestMessageId() string { + if x != nil { + return x.AcceptedRequestMessageId + } + return "" +} + +func (x *Acceptance) GetAcceptedRequestSequencingEventId() int64 { + if x != nil { + return x.AcceptedRequestSequencingEventId + } + return 0 +} + +func (x *Acceptance) GetAcceptedRequest() *Request { + if x != nil { + return x.AcceptedRequest + } + return nil +} + +// An update protocol message indicating that a workflow execution update has +// completed with the contained outcome. +type Response struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Meta *Meta `protobuf:"bytes,1,opt,name=meta,proto3" json:"meta,omitempty"` + Outcome *Outcome `protobuf:"bytes,2,opt,name=outcome,proto3" json:"outcome,omitempty"` +} + +func (x *Response) Reset() { + *x = Response{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Response) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Response) ProtoMessage() {} + +func (x *Response) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_update_v1_message_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Response.ProtoReflect.Descriptor instead. +func (*Response) Descriptor() ([]byte, []int) { + return file_temporal_api_update_v1_message_proto_rawDescGZIP(), []int{8} +} + +func (x *Response) GetMeta() *Meta { + if x != nil { + return x.Meta + } + return nil +} + +func (x *Response) GetOutcome() *Outcome { + if x != nil { + return x.Outcome + } + return nil +} + +var File_temporal_api_update_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_update_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x75, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x16, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x1a, 0x24, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, + 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x22, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, + 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x1a, 0x25, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x66, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x77, 0x0a, 0x0a, 0x57, 0x61, 0x69, 0x74, 0x50, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x12, 0x69, 0x0a, 0x0f, 0x6c, 0x69, 0x66, 0x65, 0x63, 0x79, 0x63, 0x6c, 0x65, + 0x5f, 0x73, 0x74, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x3c, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x66, 0x65, 0x63, 0x79, 0x63, + 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x52, 0x0e, 0x6c, 0x69, 0x66, 0x65, 0x63, 0x79, 0x63, 0x6c, + 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8a, 0x01, 0x0a, 0x09, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x66, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x1f, 0x0a, 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x22, 0x96, 0x01, 0x0a, 0x07, 0x4f, 0x75, 0x74, 0x63, 0x6f, 0x6d, 0x65, 0x12, 0x40, 0x0a, 0x07, 0x73, + 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x48, 0x00, 0x52, 0x07, + 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x40, 0x0a, 0x07, 0x66, 0x61, + 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x48, 0x00, 0x52, 0x07, 0x66, 0x61, + 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x42, 0x07, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x22, 0x47, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x1f, 0x0a, 0x09, 0x75, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x95, 0x01, 0x0a, 0x05, 0x49, 0x6e, 0x70, 0x75, + 0x74, 0x12, 0x3a, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, + 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x16, 0x0a, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x38, 0x0a, 0x04, 0x61, 0x72, 0x67, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x04, + 0x61, 0x72, 0x67, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x78, 0x0a, 0x07, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x34, 0x0a, 0x04, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x04, 0x6d, 0x65, + 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, 0x37, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x70, + 0x75, 0x74, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0xb2, 0x02, 0x0a, + 0x09, 0x52, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x41, 0x0a, 0x1b, 0x72, 0x65, 0x6a, + 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x72, 0x65, + 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x52, 0x0a, 0x24, 0x72, 0x65, 0x6a, + 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x73, 0x65, 0x71, + 0x75, 0x65, 0x6e, 0x63, 0x69, 0x6e, 0x67, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x20, 0x72, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x69, 0x6e, 0x67, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4e, 0x0a, 0x10, 0x72, 0x65, 0x6a, + 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x52, 0x0f, 0x72, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x22, 0xf3, 0x01, 0x0a, 0x0a, 0x41, 0x63, 0x63, 0x65, 0x70, 0x74, 0x61, 0x6e, 0x63, + 0x65, 0x12, 0x41, 0x0a, 0x1b, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x52, 0x0a, 0x24, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x5f, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x69, 0x6e, 0x67, 0x5f, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x20, 0x61, 0x63, + 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x53, 0x65, 0x71, 0x75, + 0x65, 0x6e, 0x63, 0x69, 0x6e, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x4e, 0x0a, 0x10, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x0f, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, + 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0x7f, 0x0a, 0x08, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x34, 0x0a, 0x04, 0x6d, 0x65, 0x74, 0x61, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, + 0x74, 0x61, 0x52, 0x04, 0x6d, 0x65, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x07, 0x6f, + 0x75, 0x74, 0x63, 0x6f, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x75, 0x74, 0x63, 0x6f, 0x6d, 0x65, 0x52, 0x07, 0x6f, 0x75, 0x74, + 0x63, 0x6f, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x42, 0x89, 0x01, 0x0a, 0x19, 0x69, 0x6f, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, + 0x6f, 0x50, 0x01, 0x5a, 0x23, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2f, 0x76, 0x31, + 0x3b, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0xaa, 0x02, 0x18, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x56, 0x31, + 0xea, 0x02, 0x1b, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, + 0x69, 0x3a, 0x3a, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_update_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_update_v1_message_proto_rawDescData = file_temporal_api_update_v1_message_proto_rawDesc +) + +func file_temporal_api_update_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_update_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_update_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_update_v1_message_proto_rawDescData) + }) + return file_temporal_api_update_v1_message_proto_rawDescData +} + +var file_temporal_api_update_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 9) +var file_temporal_api_update_v1_message_proto_goTypes = []any{ + (*WaitPolicy)(nil), // 0: temporal.api.update.v1.WaitPolicy + (*UpdateRef)(nil), // 1: temporal.api.update.v1.UpdateRef + (*Outcome)(nil), // 2: temporal.api.update.v1.Outcome + (*Meta)(nil), // 3: temporal.api.update.v1.Meta + (*Input)(nil), // 4: temporal.api.update.v1.Input + (*Request)(nil), // 5: temporal.api.update.v1.Request + (*Rejection)(nil), // 6: temporal.api.update.v1.Rejection + (*Acceptance)(nil), // 7: temporal.api.update.v1.Acceptance + (*Response)(nil), // 8: temporal.api.update.v1.Response + (v1.UpdateWorkflowExecutionLifecycleStage)(0), // 9: temporal.api.enums.v1.UpdateWorkflowExecutionLifecycleStage + (*v11.WorkflowExecution)(nil), // 10: temporal.api.common.v1.WorkflowExecution + (*v11.Payloads)(nil), // 11: temporal.api.common.v1.Payloads + (*v12.Failure)(nil), // 12: temporal.api.failure.v1.Failure + (*v11.Header)(nil), // 13: temporal.api.common.v1.Header +} +var file_temporal_api_update_v1_message_proto_depIdxs = []int32{ + 9, // 0: temporal.api.update.v1.WaitPolicy.lifecycle_stage:type_name -> temporal.api.enums.v1.UpdateWorkflowExecutionLifecycleStage + 10, // 1: temporal.api.update.v1.UpdateRef.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 11, // 2: temporal.api.update.v1.Outcome.success:type_name -> temporal.api.common.v1.Payloads + 12, // 3: temporal.api.update.v1.Outcome.failure:type_name -> temporal.api.failure.v1.Failure + 13, // 4: temporal.api.update.v1.Input.header:type_name -> temporal.api.common.v1.Header + 11, // 5: temporal.api.update.v1.Input.args:type_name -> temporal.api.common.v1.Payloads + 3, // 6: temporal.api.update.v1.Request.meta:type_name -> temporal.api.update.v1.Meta + 4, // 7: temporal.api.update.v1.Request.input:type_name -> temporal.api.update.v1.Input + 5, // 8: temporal.api.update.v1.Rejection.rejected_request:type_name -> temporal.api.update.v1.Request + 12, // 9: temporal.api.update.v1.Rejection.failure:type_name -> temporal.api.failure.v1.Failure + 5, // 10: temporal.api.update.v1.Acceptance.accepted_request:type_name -> temporal.api.update.v1.Request + 3, // 11: temporal.api.update.v1.Response.meta:type_name -> temporal.api.update.v1.Meta + 2, // 12: temporal.api.update.v1.Response.outcome:type_name -> temporal.api.update.v1.Outcome + 13, // [13:13] is the sub-list for method output_type + 13, // [13:13] is the sub-list for method input_type + 13, // [13:13] is the sub-list for extension type_name + 13, // [13:13] is the sub-list for extension extendee + 0, // [0:13] is the sub-list for field type_name +} + +func init() { file_temporal_api_update_v1_message_proto_init() } +func file_temporal_api_update_v1_message_proto_init() { + if File_temporal_api_update_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_update_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*WaitPolicy); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_update_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*UpdateRef); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_update_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*Outcome); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_update_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*Meta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_update_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*Input); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_update_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*Request); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_update_v1_message_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*Rejection); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_update_v1_message_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*Acceptance); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_update_v1_message_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*Response); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_temporal_api_update_v1_message_proto_msgTypes[2].OneofWrappers = []any{ + (*Outcome_Success)(nil), + (*Outcome_Failure)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_update_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 9, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_update_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_update_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_update_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_update_v1_message_proto = out.File + file_temporal_api_update_v1_message_proto_rawDesc = nil + file_temporal_api_update_v1_message_proto_goTypes = nil + file_temporal_api_update_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/version/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/version/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..f0764bb079f --- /dev/null +++ b/vendor/go.temporal.io/api/version/v1/message.go-helpers.pb.go @@ -0,0 +1,139 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package version + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type ReleaseInfo to the protobuf v3 wire format +func (val *ReleaseInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ReleaseInfo from the protobuf v3 wire format +func (val *ReleaseInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ReleaseInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ReleaseInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ReleaseInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ReleaseInfo + switch t := that.(type) { + case *ReleaseInfo: + that1 = t + case ReleaseInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type Alert to the protobuf v3 wire format +func (val *Alert) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type Alert from the protobuf v3 wire format +func (val *Alert) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *Alert) Size() int { + return proto.Size(val) +} + +// Equal returns whether two Alert values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *Alert) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *Alert + switch t := that.(type) { + case *Alert: + that1 = t + case Alert: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type VersionInfo to the protobuf v3 wire format +func (val *VersionInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type VersionInfo from the protobuf v3 wire format +func (val *VersionInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *VersionInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two VersionInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *VersionInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *VersionInfo + switch t := that.(type) { + case *VersionInfo: + that1 = t + case VersionInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/version/v1/message.pb.go b/vendor/go.temporal.io/api/version/v1/message.pb.go new file mode 100644 index 00000000000..5d811796da1 --- /dev/null +++ b/vendor/go.temporal.io/api/version/v1/message.pb.go @@ -0,0 +1,400 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/version/v1/message.proto + +package version + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/enums/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// ReleaseInfo contains information about specific version of temporal. +type ReleaseInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Version string `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"` + ReleaseTime *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=release_time,json=releaseTime,proto3" json:"release_time,omitempty"` + Notes string `protobuf:"bytes,3,opt,name=notes,proto3" json:"notes,omitempty"` +} + +func (x *ReleaseInfo) Reset() { + *x = ReleaseInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_version_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReleaseInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReleaseInfo) ProtoMessage() {} + +func (x *ReleaseInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_version_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ReleaseInfo.ProtoReflect.Descriptor instead. +func (*ReleaseInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_version_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *ReleaseInfo) GetVersion() string { + if x != nil { + return x.Version + } + return "" +} + +func (x *ReleaseInfo) GetReleaseTime() *timestamppb.Timestamp { + if x != nil { + return x.ReleaseTime + } + return nil +} + +func (x *ReleaseInfo) GetNotes() string { + if x != nil { + return x.Notes + } + return "" +} + +// Alert contains notification and severity. +type Alert struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Message string `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"` + Severity v1.Severity `protobuf:"varint,2,opt,name=severity,proto3,enum=temporal.api.enums.v1.Severity" json:"severity,omitempty"` +} + +func (x *Alert) Reset() { + *x = Alert{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_version_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Alert) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Alert) ProtoMessage() {} + +func (x *Alert) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_version_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Alert.ProtoReflect.Descriptor instead. +func (*Alert) Descriptor() ([]byte, []int) { + return file_temporal_api_version_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *Alert) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + +func (x *Alert) GetSeverity() v1.Severity { + if x != nil { + return x.Severity + } + return v1.Severity(0) +} + +// VersionInfo contains details about current and recommended release versions as well as alerts and upgrade instructions. +type VersionInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Current *ReleaseInfo `protobuf:"bytes,1,opt,name=current,proto3" json:"current,omitempty"` + Recommended *ReleaseInfo `protobuf:"bytes,2,opt,name=recommended,proto3" json:"recommended,omitempty"` + Instructions string `protobuf:"bytes,3,opt,name=instructions,proto3" json:"instructions,omitempty"` + Alerts []*Alert `protobuf:"bytes,4,rep,name=alerts,proto3" json:"alerts,omitempty"` + LastUpdateTime *timestamppb.Timestamp `protobuf:"bytes,5,opt,name=last_update_time,json=lastUpdateTime,proto3" json:"last_update_time,omitempty"` +} + +func (x *VersionInfo) Reset() { + *x = VersionInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_version_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *VersionInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VersionInfo) ProtoMessage() {} + +func (x *VersionInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_version_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use VersionInfo.ProtoReflect.Descriptor instead. +func (*VersionInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_version_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *VersionInfo) GetCurrent() *ReleaseInfo { + if x != nil { + return x.Current + } + return nil +} + +func (x *VersionInfo) GetRecommended() *ReleaseInfo { + if x != nil { + return x.Recommended + } + return nil +} + +func (x *VersionInfo) GetInstructions() string { + if x != nil { + return x.Instructions + } + return "" +} + +func (x *VersionInfo) GetAlerts() []*Alert { + if x != nil { + return x.Alerts + } + return nil +} + +func (x *VersionInfo) GetLastUpdateTime() *timestamppb.Timestamp { + if x != nil { + return x.LastUpdateTime + } + return nil +} + +var File_temporal_api_version_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_version_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x25, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x17, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x1a, 0x1f, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x22, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, + 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x88, + 0x01, 0x0a, 0x0b, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1c, 0x0a, + 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x0c, 0x72, 0x65, + 0x6c, 0x65, 0x61, 0x73, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0b, 0x72, 0x65, 0x6c, 0x65, 0x61, + 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x6e, 0x6f, + 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6e, 0x6f, 0x74, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x66, 0x0a, 0x05, 0x41, 0x6c, 0x65, 0x72, 0x74, 0x12, 0x1c, 0x0a, 0x07, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, 0x0a, 0x08, 0x73, 0x65, 0x76, 0x65, 0x72, + 0x69, 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, + 0x2e, 0x53, 0x65, 0x76, 0x65, 0x72, 0x69, 0x74, 0x79, 0x52, 0x08, 0x73, 0x65, 0x76, 0x65, 0x72, 0x69, + 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0xcb, 0x02, 0x0a, 0x0b, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x42, 0x0a, 0x07, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, + 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x63, 0x75, 0x72, 0x72, 0x65, + 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x0b, 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x6d, 0x65, + 0x6e, 0x64, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0b, + 0x72, 0x65, 0x63, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x26, + 0x0a, 0x0c, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x61, 0x6c, 0x65, 0x72, 0x74, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x6c, 0x65, + 0x72, 0x74, 0x52, 0x06, 0x61, 0x6c, 0x65, 0x72, 0x74, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x48, 0x0a, + 0x10, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x42, 0x8e, 0x01, 0x0a, 0x1a, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, + 0x01, 0x5a, 0x25, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, + 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x3b, 0x76, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0xaa, 0x02, 0x19, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x56, 0x31, + 0xea, 0x02, 0x1c, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, + 0x70, 0x69, 0x3a, 0x3a, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_version_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_version_v1_message_proto_rawDescData = file_temporal_api_version_v1_message_proto_rawDesc +) + +func file_temporal_api_version_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_version_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_version_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_version_v1_message_proto_rawDescData) + }) + return file_temporal_api_version_v1_message_proto_rawDescData +} + +var file_temporal_api_version_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 3) +var file_temporal_api_version_v1_message_proto_goTypes = []any{ + (*ReleaseInfo)(nil), // 0: temporal.api.version.v1.ReleaseInfo + (*Alert)(nil), // 1: temporal.api.version.v1.Alert + (*VersionInfo)(nil), // 2: temporal.api.version.v1.VersionInfo + (*timestamppb.Timestamp)(nil), // 3: google.protobuf.Timestamp + (v1.Severity)(0), // 4: temporal.api.enums.v1.Severity +} +var file_temporal_api_version_v1_message_proto_depIdxs = []int32{ + 3, // 0: temporal.api.version.v1.ReleaseInfo.release_time:type_name -> google.protobuf.Timestamp + 4, // 1: temporal.api.version.v1.Alert.severity:type_name -> temporal.api.enums.v1.Severity + 0, // 2: temporal.api.version.v1.VersionInfo.current:type_name -> temporal.api.version.v1.ReleaseInfo + 0, // 3: temporal.api.version.v1.VersionInfo.recommended:type_name -> temporal.api.version.v1.ReleaseInfo + 1, // 4: temporal.api.version.v1.VersionInfo.alerts:type_name -> temporal.api.version.v1.Alert + 3, // 5: temporal.api.version.v1.VersionInfo.last_update_time:type_name -> google.protobuf.Timestamp + 6, // [6:6] is the sub-list for method output_type + 6, // [6:6] is the sub-list for method input_type + 6, // [6:6] is the sub-list for extension type_name + 6, // [6:6] is the sub-list for extension extendee + 0, // [0:6] is the sub-list for field type_name +} + +func init() { file_temporal_api_version_v1_message_proto_init() } +func file_temporal_api_version_v1_message_proto_init() { + if File_temporal_api_version_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_version_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*ReleaseInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_version_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*Alert); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_version_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*VersionInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_version_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 3, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_version_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_version_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_version_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_version_v1_message_proto = out.File + file_temporal_api_version_v1_message_proto_rawDesc = nil + file_temporal_api_version_v1_message_proto_goTypes = nil + file_temporal_api_version_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/workflow/v1/message.go-helpers.pb.go b/vendor/go.temporal.io/api/workflow/v1/message.go-helpers.pb.go new file mode 100644 index 00000000000..e80f57a2e7e --- /dev/null +++ b/vendor/go.temporal.io/api/workflow/v1/message.go-helpers.pb.go @@ -0,0 +1,435 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package workflow + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type WorkflowExecutionInfo to the protobuf v3 wire format +func (val *WorkflowExecutionInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionInfo from the protobuf v3 wire format +func (val *WorkflowExecutionInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionInfo + switch t := that.(type) { + case *WorkflowExecutionInfo: + that1 = t + case WorkflowExecutionInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type WorkflowExecutionConfig to the protobuf v3 wire format +func (val *WorkflowExecutionConfig) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type WorkflowExecutionConfig from the protobuf v3 wire format +func (val *WorkflowExecutionConfig) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *WorkflowExecutionConfig) Size() int { + return proto.Size(val) +} + +// Equal returns whether two WorkflowExecutionConfig values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *WorkflowExecutionConfig) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *WorkflowExecutionConfig + switch t := that.(type) { + case *WorkflowExecutionConfig: + that1 = t + case WorkflowExecutionConfig: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PendingActivityInfo to the protobuf v3 wire format +func (val *PendingActivityInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PendingActivityInfo from the protobuf v3 wire format +func (val *PendingActivityInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PendingActivityInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PendingActivityInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PendingActivityInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PendingActivityInfo + switch t := that.(type) { + case *PendingActivityInfo: + that1 = t + case PendingActivityInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PendingChildExecutionInfo to the protobuf v3 wire format +func (val *PendingChildExecutionInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PendingChildExecutionInfo from the protobuf v3 wire format +func (val *PendingChildExecutionInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PendingChildExecutionInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PendingChildExecutionInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PendingChildExecutionInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PendingChildExecutionInfo + switch t := that.(type) { + case *PendingChildExecutionInfo: + that1 = t + case PendingChildExecutionInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PendingWorkflowTaskInfo to the protobuf v3 wire format +func (val *PendingWorkflowTaskInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PendingWorkflowTaskInfo from the protobuf v3 wire format +func (val *PendingWorkflowTaskInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PendingWorkflowTaskInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PendingWorkflowTaskInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PendingWorkflowTaskInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PendingWorkflowTaskInfo + switch t := that.(type) { + case *PendingWorkflowTaskInfo: + that1 = t + case PendingWorkflowTaskInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ResetPoints to the protobuf v3 wire format +func (val *ResetPoints) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ResetPoints from the protobuf v3 wire format +func (val *ResetPoints) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ResetPoints) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ResetPoints values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ResetPoints) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ResetPoints + switch t := that.(type) { + case *ResetPoints: + that1 = t + case ResetPoints: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ResetPointInfo to the protobuf v3 wire format +func (val *ResetPointInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ResetPointInfo from the protobuf v3 wire format +func (val *ResetPointInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ResetPointInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ResetPointInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ResetPointInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ResetPointInfo + switch t := that.(type) { + case *ResetPointInfo: + that1 = t + case ResetPointInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NewWorkflowExecutionInfo to the protobuf v3 wire format +func (val *NewWorkflowExecutionInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NewWorkflowExecutionInfo from the protobuf v3 wire format +func (val *NewWorkflowExecutionInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NewWorkflowExecutionInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NewWorkflowExecutionInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NewWorkflowExecutionInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NewWorkflowExecutionInfo + switch t := that.(type) { + case *NewWorkflowExecutionInfo: + that1 = t + case NewWorkflowExecutionInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CallbackInfo to the protobuf v3 wire format +func (val *CallbackInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CallbackInfo from the protobuf v3 wire format +func (val *CallbackInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CallbackInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CallbackInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CallbackInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CallbackInfo + switch t := that.(type) { + case *CallbackInfo: + that1 = t + case CallbackInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PendingNexusOperationInfo to the protobuf v3 wire format +func (val *PendingNexusOperationInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PendingNexusOperationInfo from the protobuf v3 wire format +func (val *PendingNexusOperationInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PendingNexusOperationInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PendingNexusOperationInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PendingNexusOperationInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PendingNexusOperationInfo + switch t := that.(type) { + case *PendingNexusOperationInfo: + that1 = t + case PendingNexusOperationInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type NexusOperationCancellationInfo to the protobuf v3 wire format +func (val *NexusOperationCancellationInfo) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type NexusOperationCancellationInfo from the protobuf v3 wire format +func (val *NexusOperationCancellationInfo) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *NexusOperationCancellationInfo) Size() int { + return proto.Size(val) +} + +// Equal returns whether two NexusOperationCancellationInfo values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *NexusOperationCancellationInfo) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *NexusOperationCancellationInfo + switch t := that.(type) { + case *NexusOperationCancellationInfo: + that1 = t + case NexusOperationCancellationInfo: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/workflow/v1/message.pb.go b/vendor/go.temporal.io/api/workflow/v1/message.pb.go new file mode 100644 index 00000000000..726c4b1e819 --- /dev/null +++ b/vendor/go.temporal.io/api/workflow/v1/message.pb.go @@ -0,0 +1,2307 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/workflow/v1/message.proto + +package workflow + +import ( + reflect "reflect" + sync "sync" + + v1 "go.temporal.io/api/common/v1" + v11 "go.temporal.io/api/enums/v1" + v14 "go.temporal.io/api/failure/v1" + v13 "go.temporal.io/api/sdk/v1" + v12 "go.temporal.io/api/taskqueue/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + durationpb "google.golang.org/protobuf/types/known/durationpb" + emptypb "google.golang.org/protobuf/types/known/emptypb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type WorkflowExecutionInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Execution *v1.WorkflowExecution `protobuf:"bytes,1,opt,name=execution,proto3" json:"execution,omitempty"` + Type *v1.WorkflowType `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` + StartTime *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + CloseTime *timestamppb.Timestamp `protobuf:"bytes,4,opt,name=close_time,json=closeTime,proto3" json:"close_time,omitempty"` + Status v11.WorkflowExecutionStatus `protobuf:"varint,5,opt,name=status,proto3,enum=temporal.api.enums.v1.WorkflowExecutionStatus" json:"status,omitempty"` + HistoryLength int64 `protobuf:"varint,6,opt,name=history_length,json=historyLength,proto3" json:"history_length,omitempty"` + ParentNamespaceId string `protobuf:"bytes,7,opt,name=parent_namespace_id,json=parentNamespaceId,proto3" json:"parent_namespace_id,omitempty"` + ParentExecution *v1.WorkflowExecution `protobuf:"bytes,8,opt,name=parent_execution,json=parentExecution,proto3" json:"parent_execution,omitempty"` + ExecutionTime *timestamppb.Timestamp `protobuf:"bytes,9,opt,name=execution_time,json=executionTime,proto3" json:"execution_time,omitempty"` + Memo *v1.Memo `protobuf:"bytes,10,opt,name=memo,proto3" json:"memo,omitempty"` + SearchAttributes *v1.SearchAttributes `protobuf:"bytes,11,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` + AutoResetPoints *ResetPoints `protobuf:"bytes,12,opt,name=auto_reset_points,json=autoResetPoints,proto3" json:"auto_reset_points,omitempty"` + TaskQueue string `protobuf:"bytes,13,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + StateTransitionCount int64 `protobuf:"varint,14,opt,name=state_transition_count,json=stateTransitionCount,proto3" json:"state_transition_count,omitempty"` + HistorySizeBytes int64 `protobuf:"varint,15,opt,name=history_size_bytes,json=historySizeBytes,proto3" json:"history_size_bytes,omitempty"` + // If set, the most recent worker version stamp that appeared in a workflow task completion + MostRecentWorkerVersionStamp *v1.WorkerVersionStamp `protobuf:"bytes,16,opt,name=most_recent_worker_version_stamp,json=mostRecentWorkerVersionStamp,proto3" json:"most_recent_worker_version_stamp,omitempty"` + // Workflow execution duration is defined as difference between close time and execution time. + // This field is only populated if the workflow is closed. + ExecutionDuration *durationpb.Duration `protobuf:"bytes,17,opt,name=execution_duration,json=executionDuration,proto3" json:"execution_duration,omitempty"` + // Contains information about the root workflow execution. + // The root workflow execution is defined as follows: + // 1. A workflow without parent workflow is its own root workflow. + // 2. A workflow that has a parent workflow has the same root workflow as its parent workflow. + // Note: workflows continued as new or reseted may or may not have parents, check examples below. + // + // Examples: + // + // Scenario 1: Workflow W1 starts child workflow W2, and W2 starts child workflow W3. + // - The root workflow of all three workflows is W1. + // Scenario 2: Workflow W1 starts child workflow W2, and W2 continued as new W3. + // - The root workflow of all three workflows is W1. + // Scenario 3: Workflow W1 continued as new W2. + // - The root workflow of W1 is W1 and the root workflow of W2 is W2. + // Scenario 4: Workflow W1 starts child workflow W2, and W2 is reseted, creating W3 + // - The root workflow of all three workflows is W1. + // Scenario 5: Workflow W1 is reseted, creating W2. + // - The root workflow of W1 is W1 and the root workflow of W2 is W2. + RootExecution *v1.WorkflowExecution `protobuf:"bytes,18,opt,name=root_execution,json=rootExecution,proto3" json:"root_execution,omitempty"` + // The currently assigned build ID for this execution. Presence of this value means worker versioning is used + // for this execution. Assigned build ID is selected based on Worker Versioning Assignment Rules + // when the first workflow task of the execution is scheduled. If the first workflow task fails and is scheduled + // again, the assigned build ID may change according to the latest versioning rules. + // Assigned build ID can also change in the middle of a execution if Compatible Redirect Rules are applied to + // this execution. + AssignedBuildId string `protobuf:"bytes,19,opt,name=assigned_build_id,json=assignedBuildId,proto3" json:"assigned_build_id,omitempty"` + // Build ID inherited from a previous/parent execution. If present, assigned_build_id will be set to this, instead + // of using the assignment rules. + InheritedBuildId string `protobuf:"bytes,20,opt,name=inherited_build_id,json=inheritedBuildId,proto3" json:"inherited_build_id,omitempty"` + // The first run ID in the execution chain. + // Executions created via the following operations are considered to be in the same chain + // - ContinueAsNew + // - Workflow Retry + // - Workflow Reset + // - Cron Schedule + FirstRunId string `protobuf:"bytes,21,opt,name=first_run_id,json=firstRunId,proto3" json:"first_run_id,omitempty"` +} + +func (x *WorkflowExecutionInfo) Reset() { + *x = WorkflowExecutionInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionInfo) ProtoMessage() {} + +func (x *WorkflowExecutionInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionInfo.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_workflow_v1_message_proto_rawDescGZIP(), []int{0} +} + +func (x *WorkflowExecutionInfo) GetExecution() *v1.WorkflowExecution { + if x != nil { + return x.Execution + } + return nil +} + +func (x *WorkflowExecutionInfo) GetType() *v1.WorkflowType { + if x != nil { + return x.Type + } + return nil +} + +func (x *WorkflowExecutionInfo) GetStartTime() *timestamppb.Timestamp { + if x != nil { + return x.StartTime + } + return nil +} + +func (x *WorkflowExecutionInfo) GetCloseTime() *timestamppb.Timestamp { + if x != nil { + return x.CloseTime + } + return nil +} + +func (x *WorkflowExecutionInfo) GetStatus() v11.WorkflowExecutionStatus { + if x != nil { + return x.Status + } + return v11.WorkflowExecutionStatus(0) +} + +func (x *WorkflowExecutionInfo) GetHistoryLength() int64 { + if x != nil { + return x.HistoryLength + } + return 0 +} + +func (x *WorkflowExecutionInfo) GetParentNamespaceId() string { + if x != nil { + return x.ParentNamespaceId + } + return "" +} + +func (x *WorkflowExecutionInfo) GetParentExecution() *v1.WorkflowExecution { + if x != nil { + return x.ParentExecution + } + return nil +} + +func (x *WorkflowExecutionInfo) GetExecutionTime() *timestamppb.Timestamp { + if x != nil { + return x.ExecutionTime + } + return nil +} + +func (x *WorkflowExecutionInfo) GetMemo() *v1.Memo { + if x != nil { + return x.Memo + } + return nil +} + +func (x *WorkflowExecutionInfo) GetSearchAttributes() *v1.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +func (x *WorkflowExecutionInfo) GetAutoResetPoints() *ResetPoints { + if x != nil { + return x.AutoResetPoints + } + return nil +} + +func (x *WorkflowExecutionInfo) GetTaskQueue() string { + if x != nil { + return x.TaskQueue + } + return "" +} + +func (x *WorkflowExecutionInfo) GetStateTransitionCount() int64 { + if x != nil { + return x.StateTransitionCount + } + return 0 +} + +func (x *WorkflowExecutionInfo) GetHistorySizeBytes() int64 { + if x != nil { + return x.HistorySizeBytes + } + return 0 +} + +func (x *WorkflowExecutionInfo) GetMostRecentWorkerVersionStamp() *v1.WorkerVersionStamp { + if x != nil { + return x.MostRecentWorkerVersionStamp + } + return nil +} + +func (x *WorkflowExecutionInfo) GetExecutionDuration() *durationpb.Duration { + if x != nil { + return x.ExecutionDuration + } + return nil +} + +func (x *WorkflowExecutionInfo) GetRootExecution() *v1.WorkflowExecution { + if x != nil { + return x.RootExecution + } + return nil +} + +func (x *WorkflowExecutionInfo) GetAssignedBuildId() string { + if x != nil { + return x.AssignedBuildId + } + return "" +} + +func (x *WorkflowExecutionInfo) GetInheritedBuildId() string { + if x != nil { + return x.InheritedBuildId + } + return "" +} + +func (x *WorkflowExecutionInfo) GetFirstRunId() string { + if x != nil { + return x.FirstRunId + } + return "" +} + +type WorkflowExecutionConfig struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TaskQueue *v12.TaskQueue `protobuf:"bytes,1,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + WorkflowExecutionTimeout *durationpb.Duration `protobuf:"bytes,2,opt,name=workflow_execution_timeout,json=workflowExecutionTimeout,proto3" json:"workflow_execution_timeout,omitempty"` + WorkflowRunTimeout *durationpb.Duration `protobuf:"bytes,3,opt,name=workflow_run_timeout,json=workflowRunTimeout,proto3" json:"workflow_run_timeout,omitempty"` + DefaultWorkflowTaskTimeout *durationpb.Duration `protobuf:"bytes,4,opt,name=default_workflow_task_timeout,json=defaultWorkflowTaskTimeout,proto3" json:"default_workflow_task_timeout,omitempty"` + // User metadata provided on start workflow. + UserMetadata *v13.UserMetadata `protobuf:"bytes,5,opt,name=user_metadata,json=userMetadata,proto3" json:"user_metadata,omitempty"` +} + +func (x *WorkflowExecutionConfig) Reset() { + *x = WorkflowExecutionConfig{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowExecutionConfig) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowExecutionConfig) ProtoMessage() {} + +func (x *WorkflowExecutionConfig) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowExecutionConfig.ProtoReflect.Descriptor instead. +func (*WorkflowExecutionConfig) Descriptor() ([]byte, []int) { + return file_temporal_api_workflow_v1_message_proto_rawDescGZIP(), []int{1} +} + +func (x *WorkflowExecutionConfig) GetTaskQueue() *v12.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *WorkflowExecutionConfig) GetWorkflowExecutionTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowExecutionTimeout + } + return nil +} + +func (x *WorkflowExecutionConfig) GetWorkflowRunTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowRunTimeout + } + return nil +} + +func (x *WorkflowExecutionConfig) GetDefaultWorkflowTaskTimeout() *durationpb.Duration { + if x != nil { + return x.DefaultWorkflowTaskTimeout + } + return nil +} + +func (x *WorkflowExecutionConfig) GetUserMetadata() *v13.UserMetadata { + if x != nil { + return x.UserMetadata + } + return nil +} + +type PendingActivityInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ActivityId string `protobuf:"bytes,1,opt,name=activity_id,json=activityId,proto3" json:"activity_id,omitempty"` + ActivityType *v1.ActivityType `protobuf:"bytes,2,opt,name=activity_type,json=activityType,proto3" json:"activity_type,omitempty"` + State v11.PendingActivityState `protobuf:"varint,3,opt,name=state,proto3,enum=temporal.api.enums.v1.PendingActivityState" json:"state,omitempty"` + HeartbeatDetails *v1.Payloads `protobuf:"bytes,4,opt,name=heartbeat_details,json=heartbeatDetails,proto3" json:"heartbeat_details,omitempty"` + LastHeartbeatTime *timestamppb.Timestamp `protobuf:"bytes,5,opt,name=last_heartbeat_time,json=lastHeartbeatTime,proto3" json:"last_heartbeat_time,omitempty"` + LastStartedTime *timestamppb.Timestamp `protobuf:"bytes,6,opt,name=last_started_time,json=lastStartedTime,proto3" json:"last_started_time,omitempty"` + Attempt int32 `protobuf:"varint,7,opt,name=attempt,proto3" json:"attempt,omitempty"` + MaximumAttempts int32 `protobuf:"varint,8,opt,name=maximum_attempts,json=maximumAttempts,proto3" json:"maximum_attempts,omitempty"` + ScheduledTime *timestamppb.Timestamp `protobuf:"bytes,9,opt,name=scheduled_time,json=scheduledTime,proto3" json:"scheduled_time,omitempty"` + ExpirationTime *timestamppb.Timestamp `protobuf:"bytes,10,opt,name=expiration_time,json=expirationTime,proto3" json:"expiration_time,omitempty"` + LastFailure *v14.Failure `protobuf:"bytes,11,opt,name=last_failure,json=lastFailure,proto3" json:"last_failure,omitempty"` + LastWorkerIdentity string `protobuf:"bytes,12,opt,name=last_worker_identity,json=lastWorkerIdentity,proto3" json:"last_worker_identity,omitempty"` + // Absence of `assigned_build_id` generally means this task is on an "unversioned" task queue. + // In rare cases, it can also mean that the task queue is versioned but we failed to write activity's + // independently-assigned build ID to the database. This case heals automatically once the task is dispatched. + // + // Types that are assignable to AssignedBuildId: + // + // *PendingActivityInfo_UseWorkflowBuildId + // *PendingActivityInfo_LastIndependentlyAssignedBuildId + AssignedBuildId isPendingActivityInfo_AssignedBuildId `protobuf_oneof:"assigned_build_id"` + // The version stamp of the worker to whom this activity was most recently dispatched + LastWorkerVersionStamp *v1.WorkerVersionStamp `protobuf:"bytes,15,opt,name=last_worker_version_stamp,json=lastWorkerVersionStamp,proto3" json:"last_worker_version_stamp,omitempty"` +} + +func (x *PendingActivityInfo) Reset() { + *x = PendingActivityInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PendingActivityInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PendingActivityInfo) ProtoMessage() {} + +func (x *PendingActivityInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PendingActivityInfo.ProtoReflect.Descriptor instead. +func (*PendingActivityInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_workflow_v1_message_proto_rawDescGZIP(), []int{2} +} + +func (x *PendingActivityInfo) GetActivityId() string { + if x != nil { + return x.ActivityId + } + return "" +} + +func (x *PendingActivityInfo) GetActivityType() *v1.ActivityType { + if x != nil { + return x.ActivityType + } + return nil +} + +func (x *PendingActivityInfo) GetState() v11.PendingActivityState { + if x != nil { + return x.State + } + return v11.PendingActivityState(0) +} + +func (x *PendingActivityInfo) GetHeartbeatDetails() *v1.Payloads { + if x != nil { + return x.HeartbeatDetails + } + return nil +} + +func (x *PendingActivityInfo) GetLastHeartbeatTime() *timestamppb.Timestamp { + if x != nil { + return x.LastHeartbeatTime + } + return nil +} + +func (x *PendingActivityInfo) GetLastStartedTime() *timestamppb.Timestamp { + if x != nil { + return x.LastStartedTime + } + return nil +} + +func (x *PendingActivityInfo) GetAttempt() int32 { + if x != nil { + return x.Attempt + } + return 0 +} + +func (x *PendingActivityInfo) GetMaximumAttempts() int32 { + if x != nil { + return x.MaximumAttempts + } + return 0 +} + +func (x *PendingActivityInfo) GetScheduledTime() *timestamppb.Timestamp { + if x != nil { + return x.ScheduledTime + } + return nil +} + +func (x *PendingActivityInfo) GetExpirationTime() *timestamppb.Timestamp { + if x != nil { + return x.ExpirationTime + } + return nil +} + +func (x *PendingActivityInfo) GetLastFailure() *v14.Failure { + if x != nil { + return x.LastFailure + } + return nil +} + +func (x *PendingActivityInfo) GetLastWorkerIdentity() string { + if x != nil { + return x.LastWorkerIdentity + } + return "" +} + +func (m *PendingActivityInfo) GetAssignedBuildId() isPendingActivityInfo_AssignedBuildId { + if m != nil { + return m.AssignedBuildId + } + return nil +} + +func (x *PendingActivityInfo) GetUseWorkflowBuildId() *emptypb.Empty { + if x, ok := x.GetAssignedBuildId().(*PendingActivityInfo_UseWorkflowBuildId); ok { + return x.UseWorkflowBuildId + } + return nil +} + +func (x *PendingActivityInfo) GetLastIndependentlyAssignedBuildId() string { + if x, ok := x.GetAssignedBuildId().(*PendingActivityInfo_LastIndependentlyAssignedBuildId); ok { + return x.LastIndependentlyAssignedBuildId + } + return "" +} + +func (x *PendingActivityInfo) GetLastWorkerVersionStamp() *v1.WorkerVersionStamp { + if x != nil { + return x.LastWorkerVersionStamp + } + return nil +} + +type isPendingActivityInfo_AssignedBuildId interface { + isPendingActivityInfo_AssignedBuildId() +} + +type PendingActivityInfo_UseWorkflowBuildId struct { + // When present, it means this activity is assigned to the build ID of its workflow. + UseWorkflowBuildId *emptypb.Empty `protobuf:"bytes,13,opt,name=use_workflow_build_id,json=useWorkflowBuildId,proto3,oneof"` +} + +type PendingActivityInfo_LastIndependentlyAssignedBuildId struct { + // This means the activity is independently versioned and not bound to the build ID of its workflow. + // The activity will use the build id in this field instead. + // If the task fails and is scheduled again, the assigned build ID may change according to the latest versioning + // rules. + LastIndependentlyAssignedBuildId string `protobuf:"bytes,14,opt,name=last_independently_assigned_build_id,json=lastIndependentlyAssignedBuildId,proto3,oneof"` +} + +func (*PendingActivityInfo_UseWorkflowBuildId) isPendingActivityInfo_AssignedBuildId() {} + +func (*PendingActivityInfo_LastIndependentlyAssignedBuildId) isPendingActivityInfo_AssignedBuildId() { +} + +type PendingChildExecutionInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + WorkflowId string `protobuf:"bytes,1,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + RunId string `protobuf:"bytes,2,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` + WorkflowTypeName string `protobuf:"bytes,3,opt,name=workflow_type_name,json=workflowTypeName,proto3" json:"workflow_type_name,omitempty"` + InitiatedId int64 `protobuf:"varint,4,opt,name=initiated_id,json=initiatedId,proto3" json:"initiated_id,omitempty"` + // Default: PARENT_CLOSE_POLICY_TERMINATE. + ParentClosePolicy v11.ParentClosePolicy `protobuf:"varint,5,opt,name=parent_close_policy,json=parentClosePolicy,proto3,enum=temporal.api.enums.v1.ParentClosePolicy" json:"parent_close_policy,omitempty"` +} + +func (x *PendingChildExecutionInfo) Reset() { + *x = PendingChildExecutionInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PendingChildExecutionInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PendingChildExecutionInfo) ProtoMessage() {} + +func (x *PendingChildExecutionInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PendingChildExecutionInfo.ProtoReflect.Descriptor instead. +func (*PendingChildExecutionInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_workflow_v1_message_proto_rawDescGZIP(), []int{3} +} + +func (x *PendingChildExecutionInfo) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *PendingChildExecutionInfo) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +func (x *PendingChildExecutionInfo) GetWorkflowTypeName() string { + if x != nil { + return x.WorkflowTypeName + } + return "" +} + +func (x *PendingChildExecutionInfo) GetInitiatedId() int64 { + if x != nil { + return x.InitiatedId + } + return 0 +} + +func (x *PendingChildExecutionInfo) GetParentClosePolicy() v11.ParentClosePolicy { + if x != nil { + return x.ParentClosePolicy + } + return v11.ParentClosePolicy(0) +} + +type PendingWorkflowTaskInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + State v11.PendingWorkflowTaskState `protobuf:"varint,1,opt,name=state,proto3,enum=temporal.api.enums.v1.PendingWorkflowTaskState" json:"state,omitempty"` + ScheduledTime *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=scheduled_time,json=scheduledTime,proto3" json:"scheduled_time,omitempty"` + // original_scheduled_time is the scheduled time of the first workflow task during workflow task heartbeat. + // Heartbeat workflow task is done by RespondWorkflowTaskComplete with ForceCreateNewWorkflowTask == true and no command + // In this case, OriginalScheduledTime won't change. Then when current time - original_scheduled_time exceeds + // some threshold, the workflow task will be forced timeout. + OriginalScheduledTime *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=original_scheduled_time,json=originalScheduledTime,proto3" json:"original_scheduled_time,omitempty"` + StartedTime *timestamppb.Timestamp `protobuf:"bytes,4,opt,name=started_time,json=startedTime,proto3" json:"started_time,omitempty"` + Attempt int32 `protobuf:"varint,5,opt,name=attempt,proto3" json:"attempt,omitempty"` +} + +func (x *PendingWorkflowTaskInfo) Reset() { + *x = PendingWorkflowTaskInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PendingWorkflowTaskInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PendingWorkflowTaskInfo) ProtoMessage() {} + +func (x *PendingWorkflowTaskInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PendingWorkflowTaskInfo.ProtoReflect.Descriptor instead. +func (*PendingWorkflowTaskInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_workflow_v1_message_proto_rawDescGZIP(), []int{4} +} + +func (x *PendingWorkflowTaskInfo) GetState() v11.PendingWorkflowTaskState { + if x != nil { + return x.State + } + return v11.PendingWorkflowTaskState(0) +} + +func (x *PendingWorkflowTaskInfo) GetScheduledTime() *timestamppb.Timestamp { + if x != nil { + return x.ScheduledTime + } + return nil +} + +func (x *PendingWorkflowTaskInfo) GetOriginalScheduledTime() *timestamppb.Timestamp { + if x != nil { + return x.OriginalScheduledTime + } + return nil +} + +func (x *PendingWorkflowTaskInfo) GetStartedTime() *timestamppb.Timestamp { + if x != nil { + return x.StartedTime + } + return nil +} + +func (x *PendingWorkflowTaskInfo) GetAttempt() int32 { + if x != nil { + return x.Attempt + } + return 0 +} + +type ResetPoints struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Points []*ResetPointInfo `protobuf:"bytes,1,rep,name=points,proto3" json:"points,omitempty"` +} + +func (x *ResetPoints) Reset() { + *x = ResetPoints{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResetPoints) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResetPoints) ProtoMessage() {} + +func (x *ResetPoints) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResetPoints.ProtoReflect.Descriptor instead. +func (*ResetPoints) Descriptor() ([]byte, []int) { + return file_temporal_api_workflow_v1_message_proto_rawDescGZIP(), []int{5} +} + +func (x *ResetPoints) GetPoints() []*ResetPointInfo { + if x != nil { + return x.Points + } + return nil +} + +// ResetPointInfo records the workflow event id that is the first one processed by a given +// build id or binary checksum. A new reset point will be created if either build id or binary +// checksum changes (although in general only one or the other will be used at a time). +type ResetPointInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Worker build id. + BuildId string `protobuf:"bytes,7,opt,name=build_id,json=buildId,proto3" json:"build_id,omitempty"` + // A worker binary version identifier (deprecated). + BinaryChecksum string `protobuf:"bytes,1,opt,name=binary_checksum,json=binaryChecksum,proto3" json:"binary_checksum,omitempty"` + // The first run ID in the execution chain that was touched by this worker build. + RunId string `protobuf:"bytes,2,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` + // Event ID of the first WorkflowTaskCompleted event processed by this worker build. + FirstWorkflowTaskCompletedId int64 `protobuf:"varint,3,opt,name=first_workflow_task_completed_id,json=firstWorkflowTaskCompletedId,proto3" json:"first_workflow_task_completed_id,omitempty"` + CreateTime *timestamppb.Timestamp `protobuf:"bytes,4,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` + // (-- api-linter: core::0214::resource-expiry=disabled + // + // aip.dev/not-precedent: TTL is not defined for ResetPointInfo. --) + // + // The time that the run is deleted due to retention. + ExpireTime *timestamppb.Timestamp `protobuf:"bytes,5,opt,name=expire_time,json=expireTime,proto3" json:"expire_time,omitempty"` + // false if the reset point has pending childWFs/reqCancels/signalExternals. + Resettable bool `protobuf:"varint,6,opt,name=resettable,proto3" json:"resettable,omitempty"` +} + +func (x *ResetPointInfo) Reset() { + *x = ResetPointInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResetPointInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResetPointInfo) ProtoMessage() {} + +func (x *ResetPointInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResetPointInfo.ProtoReflect.Descriptor instead. +func (*ResetPointInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_workflow_v1_message_proto_rawDescGZIP(), []int{6} +} + +func (x *ResetPointInfo) GetBuildId() string { + if x != nil { + return x.BuildId + } + return "" +} + +func (x *ResetPointInfo) GetBinaryChecksum() string { + if x != nil { + return x.BinaryChecksum + } + return "" +} + +func (x *ResetPointInfo) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +func (x *ResetPointInfo) GetFirstWorkflowTaskCompletedId() int64 { + if x != nil { + return x.FirstWorkflowTaskCompletedId + } + return 0 +} + +func (x *ResetPointInfo) GetCreateTime() *timestamppb.Timestamp { + if x != nil { + return x.CreateTime + } + return nil +} + +func (x *ResetPointInfo) GetExpireTime() *timestamppb.Timestamp { + if x != nil { + return x.ExpireTime + } + return nil +} + +func (x *ResetPointInfo) GetResettable() bool { + if x != nil { + return x.Resettable + } + return false +} + +// NewWorkflowExecutionInfo is a shared message that encapsulates all the +// required arguments to starting a workflow in different contexts. +type NewWorkflowExecutionInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + WorkflowId string `protobuf:"bytes,1,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + WorkflowType *v1.WorkflowType `protobuf:"bytes,2,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + TaskQueue *v12.TaskQueue `protobuf:"bytes,3,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + // Serialized arguments to the workflow. + Input *v1.Payloads `protobuf:"bytes,4,opt,name=input,proto3" json:"input,omitempty"` + // Total workflow execution timeout including retries and continue as new. + WorkflowExecutionTimeout *durationpb.Duration `protobuf:"bytes,5,opt,name=workflow_execution_timeout,json=workflowExecutionTimeout,proto3" json:"workflow_execution_timeout,omitempty"` + // Timeout of a single workflow run. + WorkflowRunTimeout *durationpb.Duration `protobuf:"bytes,6,opt,name=workflow_run_timeout,json=workflowRunTimeout,proto3" json:"workflow_run_timeout,omitempty"` + // Timeout of a single workflow task. + WorkflowTaskTimeout *durationpb.Duration `protobuf:"bytes,7,opt,name=workflow_task_timeout,json=workflowTaskTimeout,proto3" json:"workflow_task_timeout,omitempty"` + // Default: WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE. + WorkflowIdReusePolicy v11.WorkflowIdReusePolicy `protobuf:"varint,8,opt,name=workflow_id_reuse_policy,json=workflowIdReusePolicy,proto3,enum=temporal.api.enums.v1.WorkflowIdReusePolicy" json:"workflow_id_reuse_policy,omitempty"` + // The retry policy for the workflow. Will never exceed `workflow_execution_timeout`. + RetryPolicy *v1.RetryPolicy `protobuf:"bytes,9,opt,name=retry_policy,json=retryPolicy,proto3" json:"retry_policy,omitempty"` + // See https://docs.temporal.io/docs/content/what-is-a-temporal-cron-job/ + CronSchedule string `protobuf:"bytes,10,opt,name=cron_schedule,json=cronSchedule,proto3" json:"cron_schedule,omitempty"` + Memo *v1.Memo `protobuf:"bytes,11,opt,name=memo,proto3" json:"memo,omitempty"` + SearchAttributes *v1.SearchAttributes `protobuf:"bytes,12,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` + Header *v1.Header `protobuf:"bytes,13,opt,name=header,proto3" json:"header,omitempty"` + // Metadata on the workflow if it is started. This is carried over to the WorkflowExecutionConfig + // for use by user interfaces to display the fixed as-of-start summary and details of the + // workflow. + UserMetadata *v13.UserMetadata `protobuf:"bytes,14,opt,name=user_metadata,json=userMetadata,proto3" json:"user_metadata,omitempty"` +} + +func (x *NewWorkflowExecutionInfo) Reset() { + *x = NewWorkflowExecutionInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NewWorkflowExecutionInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NewWorkflowExecutionInfo) ProtoMessage() {} + +func (x *NewWorkflowExecutionInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NewWorkflowExecutionInfo.ProtoReflect.Descriptor instead. +func (*NewWorkflowExecutionInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_workflow_v1_message_proto_rawDescGZIP(), []int{7} +} + +func (x *NewWorkflowExecutionInfo) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *NewWorkflowExecutionInfo) GetWorkflowType() *v1.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *NewWorkflowExecutionInfo) GetTaskQueue() *v12.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *NewWorkflowExecutionInfo) GetInput() *v1.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *NewWorkflowExecutionInfo) GetWorkflowExecutionTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowExecutionTimeout + } + return nil +} + +func (x *NewWorkflowExecutionInfo) GetWorkflowRunTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowRunTimeout + } + return nil +} + +func (x *NewWorkflowExecutionInfo) GetWorkflowTaskTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowTaskTimeout + } + return nil +} + +func (x *NewWorkflowExecutionInfo) GetWorkflowIdReusePolicy() v11.WorkflowIdReusePolicy { + if x != nil { + return x.WorkflowIdReusePolicy + } + return v11.WorkflowIdReusePolicy(0) +} + +func (x *NewWorkflowExecutionInfo) GetRetryPolicy() *v1.RetryPolicy { + if x != nil { + return x.RetryPolicy + } + return nil +} + +func (x *NewWorkflowExecutionInfo) GetCronSchedule() string { + if x != nil { + return x.CronSchedule + } + return "" +} + +func (x *NewWorkflowExecutionInfo) GetMemo() *v1.Memo { + if x != nil { + return x.Memo + } + return nil +} + +func (x *NewWorkflowExecutionInfo) GetSearchAttributes() *v1.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +func (x *NewWorkflowExecutionInfo) GetHeader() *v1.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *NewWorkflowExecutionInfo) GetUserMetadata() *v13.UserMetadata { + if x != nil { + return x.UserMetadata + } + return nil +} + +// CallbackInfo contains the state of an attached workflow callback. +type CallbackInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Information on how this callback should be invoked (e.g. its URL and type). + Callback *v1.Callback `protobuf:"bytes,1,opt,name=callback,proto3" json:"callback,omitempty"` + // Trigger for this callback. + Trigger *CallbackInfo_Trigger `protobuf:"bytes,2,opt,name=trigger,proto3" json:"trigger,omitempty"` + // The time when the callback was registered. + RegistrationTime *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=registration_time,json=registrationTime,proto3" json:"registration_time,omitempty"` + State v11.CallbackState `protobuf:"varint,4,opt,name=state,proto3,enum=temporal.api.enums.v1.CallbackState" json:"state,omitempty"` + // The number of attempts made to deliver the callback. + // This number represents a minimum bound since the attempt is incremented after the callback request completes. + Attempt int32 `protobuf:"varint,5,opt,name=attempt,proto3" json:"attempt,omitempty"` + // The time when the last attempt completed. + LastAttemptCompleteTime *timestamppb.Timestamp `protobuf:"bytes,6,opt,name=last_attempt_complete_time,json=lastAttemptCompleteTime,proto3" json:"last_attempt_complete_time,omitempty"` + // The last attempt's failure, if any. + LastAttemptFailure *v14.Failure `protobuf:"bytes,7,opt,name=last_attempt_failure,json=lastAttemptFailure,proto3" json:"last_attempt_failure,omitempty"` + // The time when the next attempt is scheduled. + NextAttemptScheduleTime *timestamppb.Timestamp `protobuf:"bytes,8,opt,name=next_attempt_schedule_time,json=nextAttemptScheduleTime,proto3" json:"next_attempt_schedule_time,omitempty"` +} + +func (x *CallbackInfo) Reset() { + *x = CallbackInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CallbackInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CallbackInfo) ProtoMessage() {} + +func (x *CallbackInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CallbackInfo.ProtoReflect.Descriptor instead. +func (*CallbackInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_workflow_v1_message_proto_rawDescGZIP(), []int{8} +} + +func (x *CallbackInfo) GetCallback() *v1.Callback { + if x != nil { + return x.Callback + } + return nil +} + +func (x *CallbackInfo) GetTrigger() *CallbackInfo_Trigger { + if x != nil { + return x.Trigger + } + return nil +} + +func (x *CallbackInfo) GetRegistrationTime() *timestamppb.Timestamp { + if x != nil { + return x.RegistrationTime + } + return nil +} + +func (x *CallbackInfo) GetState() v11.CallbackState { + if x != nil { + return x.State + } + return v11.CallbackState(0) +} + +func (x *CallbackInfo) GetAttempt() int32 { + if x != nil { + return x.Attempt + } + return 0 +} + +func (x *CallbackInfo) GetLastAttemptCompleteTime() *timestamppb.Timestamp { + if x != nil { + return x.LastAttemptCompleteTime + } + return nil +} + +func (x *CallbackInfo) GetLastAttemptFailure() *v14.Failure { + if x != nil { + return x.LastAttemptFailure + } + return nil +} + +func (x *CallbackInfo) GetNextAttemptScheduleTime() *timestamppb.Timestamp { + if x != nil { + return x.NextAttemptScheduleTime + } + return nil +} + +// PendingNexusOperationInfo contains the state of a pending Nexus operation. +type PendingNexusOperationInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Endpoint name. + // Resolved to a URL via the cluster's endpoint registry. + Endpoint string `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + // Service name. + Service string `protobuf:"bytes,2,opt,name=service,proto3" json:"service,omitempty"` + // Operation name. + Operation string `protobuf:"bytes,3,opt,name=operation,proto3" json:"operation,omitempty"` + // Operation ID. Only set for asynchronous operations after a successful StartOperation call. + OperationId string `protobuf:"bytes,4,opt,name=operation_id,json=operationId,proto3" json:"operation_id,omitempty"` + // Schedule-to-close timeout for this operation. + // This is the only timeout settable by a workflow. + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + ScheduleToCloseTimeout *durationpb.Duration `protobuf:"bytes,5,opt,name=schedule_to_close_timeout,json=scheduleToCloseTimeout,proto3" json:"schedule_to_close_timeout,omitempty"` + // The time when the operation was scheduled. + ScheduledTime *timestamppb.Timestamp `protobuf:"bytes,6,opt,name=scheduled_time,json=scheduledTime,proto3" json:"scheduled_time,omitempty"` + State v11.PendingNexusOperationState `protobuf:"varint,7,opt,name=state,proto3,enum=temporal.api.enums.v1.PendingNexusOperationState" json:"state,omitempty"` + // The number of attempts made to deliver the start operation request. + // This number represents a minimum bound since the attempt is incremented after the request completes. + Attempt int32 `protobuf:"varint,8,opt,name=attempt,proto3" json:"attempt,omitempty"` + // The time when the last attempt completed. + LastAttemptCompleteTime *timestamppb.Timestamp `protobuf:"bytes,9,opt,name=last_attempt_complete_time,json=lastAttemptCompleteTime,proto3" json:"last_attempt_complete_time,omitempty"` + // The last attempt's failure, if any. + LastAttemptFailure *v14.Failure `protobuf:"bytes,10,opt,name=last_attempt_failure,json=lastAttemptFailure,proto3" json:"last_attempt_failure,omitempty"` + // The time when the next attempt is scheduled. + NextAttemptScheduleTime *timestamppb.Timestamp `protobuf:"bytes,11,opt,name=next_attempt_schedule_time,json=nextAttemptScheduleTime,proto3" json:"next_attempt_schedule_time,omitempty"` + CancellationInfo *NexusOperationCancellationInfo `protobuf:"bytes,12,opt,name=cancellation_info,json=cancellationInfo,proto3" json:"cancellation_info,omitempty"` + // The event ID of the NexusOperationScheduled event. Can be used to correlate an operation in the + // DescribeWorkflowExecution response with workflow history. + ScheduledEventId int64 `protobuf:"varint,13,opt,name=scheduled_event_id,json=scheduledEventId,proto3" json:"scheduled_event_id,omitempty"` +} + +func (x *PendingNexusOperationInfo) Reset() { + *x = PendingNexusOperationInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PendingNexusOperationInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PendingNexusOperationInfo) ProtoMessage() {} + +func (x *PendingNexusOperationInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PendingNexusOperationInfo.ProtoReflect.Descriptor instead. +func (*PendingNexusOperationInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_workflow_v1_message_proto_rawDescGZIP(), []int{9} +} + +func (x *PendingNexusOperationInfo) GetEndpoint() string { + if x != nil { + return x.Endpoint + } + return "" +} + +func (x *PendingNexusOperationInfo) GetService() string { + if x != nil { + return x.Service + } + return "" +} + +func (x *PendingNexusOperationInfo) GetOperation() string { + if x != nil { + return x.Operation + } + return "" +} + +func (x *PendingNexusOperationInfo) GetOperationId() string { + if x != nil { + return x.OperationId + } + return "" +} + +func (x *PendingNexusOperationInfo) GetScheduleToCloseTimeout() *durationpb.Duration { + if x != nil { + return x.ScheduleToCloseTimeout + } + return nil +} + +func (x *PendingNexusOperationInfo) GetScheduledTime() *timestamppb.Timestamp { + if x != nil { + return x.ScheduledTime + } + return nil +} + +func (x *PendingNexusOperationInfo) GetState() v11.PendingNexusOperationState { + if x != nil { + return x.State + } + return v11.PendingNexusOperationState(0) +} + +func (x *PendingNexusOperationInfo) GetAttempt() int32 { + if x != nil { + return x.Attempt + } + return 0 +} + +func (x *PendingNexusOperationInfo) GetLastAttemptCompleteTime() *timestamppb.Timestamp { + if x != nil { + return x.LastAttemptCompleteTime + } + return nil +} + +func (x *PendingNexusOperationInfo) GetLastAttemptFailure() *v14.Failure { + if x != nil { + return x.LastAttemptFailure + } + return nil +} + +func (x *PendingNexusOperationInfo) GetNextAttemptScheduleTime() *timestamppb.Timestamp { + if x != nil { + return x.NextAttemptScheduleTime + } + return nil +} + +func (x *PendingNexusOperationInfo) GetCancellationInfo() *NexusOperationCancellationInfo { + if x != nil { + return x.CancellationInfo + } + return nil +} + +func (x *PendingNexusOperationInfo) GetScheduledEventId() int64 { + if x != nil { + return x.ScheduledEventId + } + return 0 +} + +// NexusOperationCancellationInfo contains the state of a nexus operation cancellation. +type NexusOperationCancellationInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The time when cancellation was requested. + RequestedTime *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=requested_time,json=requestedTime,proto3" json:"requested_time,omitempty"` + State v11.NexusOperationCancellationState `protobuf:"varint,2,opt,name=state,proto3,enum=temporal.api.enums.v1.NexusOperationCancellationState" json:"state,omitempty"` + // The number of attempts made to deliver the cancel operation request. + // This number represents a minimum bound since the attempt is incremented after the request completes. + Attempt int32 `protobuf:"varint,3,opt,name=attempt,proto3" json:"attempt,omitempty"` + // The time when the last attempt completed. + LastAttemptCompleteTime *timestamppb.Timestamp `protobuf:"bytes,4,opt,name=last_attempt_complete_time,json=lastAttemptCompleteTime,proto3" json:"last_attempt_complete_time,omitempty"` + // The last attempt's failure, if any. + LastAttemptFailure *v14.Failure `protobuf:"bytes,5,opt,name=last_attempt_failure,json=lastAttemptFailure,proto3" json:"last_attempt_failure,omitempty"` + // The time when the next attempt is scheduled. + NextAttemptScheduleTime *timestamppb.Timestamp `protobuf:"bytes,6,opt,name=next_attempt_schedule_time,json=nextAttemptScheduleTime,proto3" json:"next_attempt_schedule_time,omitempty"` +} + +func (x *NexusOperationCancellationInfo) Reset() { + *x = NexusOperationCancellationInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *NexusOperationCancellationInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NexusOperationCancellationInfo) ProtoMessage() {} + +func (x *NexusOperationCancellationInfo) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NexusOperationCancellationInfo.ProtoReflect.Descriptor instead. +func (*NexusOperationCancellationInfo) Descriptor() ([]byte, []int) { + return file_temporal_api_workflow_v1_message_proto_rawDescGZIP(), []int{10} +} + +func (x *NexusOperationCancellationInfo) GetRequestedTime() *timestamppb.Timestamp { + if x != nil { + return x.RequestedTime + } + return nil +} + +func (x *NexusOperationCancellationInfo) GetState() v11.NexusOperationCancellationState { + if x != nil { + return x.State + } + return v11.NexusOperationCancellationState(0) +} + +func (x *NexusOperationCancellationInfo) GetAttempt() int32 { + if x != nil { + return x.Attempt + } + return 0 +} + +func (x *NexusOperationCancellationInfo) GetLastAttemptCompleteTime() *timestamppb.Timestamp { + if x != nil { + return x.LastAttemptCompleteTime + } + return nil +} + +func (x *NexusOperationCancellationInfo) GetLastAttemptFailure() *v14.Failure { + if x != nil { + return x.LastAttemptFailure + } + return nil +} + +func (x *NexusOperationCancellationInfo) GetNextAttemptScheduleTime() *timestamppb.Timestamp { + if x != nil { + return x.NextAttemptScheduleTime + } + return nil +} + +// Trigger for when the workflow is closed. +type CallbackInfo_WorkflowClosed struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CallbackInfo_WorkflowClosed) Reset() { + *x = CallbackInfo_WorkflowClosed{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CallbackInfo_WorkflowClosed) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CallbackInfo_WorkflowClosed) ProtoMessage() {} + +func (x *CallbackInfo_WorkflowClosed) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CallbackInfo_WorkflowClosed.ProtoReflect.Descriptor instead. +func (*CallbackInfo_WorkflowClosed) Descriptor() ([]byte, []int) { + return file_temporal_api_workflow_v1_message_proto_rawDescGZIP(), []int{8, 0} +} + +type CallbackInfo_Trigger struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Variant: + // + // *CallbackInfo_Trigger_WorkflowClosed + Variant isCallbackInfo_Trigger_Variant `protobuf_oneof:"variant"` +} + +func (x *CallbackInfo_Trigger) Reset() { + *x = CallbackInfo_Trigger{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CallbackInfo_Trigger) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CallbackInfo_Trigger) ProtoMessage() {} + +func (x *CallbackInfo_Trigger) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflow_v1_message_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CallbackInfo_Trigger.ProtoReflect.Descriptor instead. +func (*CallbackInfo_Trigger) Descriptor() ([]byte, []int) { + return file_temporal_api_workflow_v1_message_proto_rawDescGZIP(), []int{8, 1} +} + +func (m *CallbackInfo_Trigger) GetVariant() isCallbackInfo_Trigger_Variant { + if m != nil { + return m.Variant + } + return nil +} + +func (x *CallbackInfo_Trigger) GetWorkflowClosed() *CallbackInfo_WorkflowClosed { + if x, ok := x.GetVariant().(*CallbackInfo_Trigger_WorkflowClosed); ok { + return x.WorkflowClosed + } + return nil +} + +type isCallbackInfo_Trigger_Variant interface { + isCallbackInfo_Trigger_Variant() +} + +type CallbackInfo_Trigger_WorkflowClosed struct { + WorkflowClosed *CallbackInfo_WorkflowClosed `protobuf:"bytes,1,opt,name=workflow_closed,json=workflowClosed,proto3,oneof"` +} + +func (*CallbackInfo_Trigger_WorkflowClosed) isCallbackInfo_Trigger_Variant() {} + +var File_temporal_api_workflow_v1_message_proto protoreflect.FileDescriptor + +var file_temporal_api_workflow_v1_message_proto_rawDesc = []byte{ + 0x0a, 0x26, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x18, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x1a, 0x1e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x65, 0x6d, 0x70, + 0x74, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x22, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x1a, 0x25, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, + 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2f, 0x76, 0x31, + 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x73, 0x64, 0x6b, 0x2f, 0x76, + 0x31, 0x2f, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x87, 0x0b, 0x0a, 0x15, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x4b, 0x0a, 0x09, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3c, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x0a, + 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x63, 0x6c, 0x6f, 0x73, 0x65, + 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x29, 0x0a, 0x0e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x5f, 0x6c, 0x65, 0x6e, 0x67, 0x74, + 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x4c, + 0x65, 0x6e, 0x67, 0x74, 0x68, 0x42, 0x02, 0x68, 0x00, 0x12, 0x32, 0x0a, 0x13, 0x70, 0x61, 0x72, 0x65, + 0x6e, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x58, 0x0a, 0x10, 0x70, 0x61, + 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0f, 0x70, 0x61, 0x72, + 0x65, 0x6e, 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x45, 0x0a, 0x0e, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x0d, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x6d, 0x6f, 0x52, 0x04, 0x6d, 0x65, + 0x6d, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, 0x11, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, + 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x10, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x55, 0x0a, 0x11, + 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, + 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, + 0x52, 0x65, 0x73, 0x65, 0x74, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x52, 0x0f, 0x61, 0x75, 0x74, 0x6f, + 0x52, 0x65, 0x73, 0x65, 0x74, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, + 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x38, 0x0a, 0x16, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x03, 0x52, 0x14, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, + 0x75, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, + 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x0f, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x10, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x79, 0x74, + 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x76, 0x0a, 0x20, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x72, 0x65, + 0x63, 0x65, 0x6e, 0x74, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x1c, 0x6d, 0x6f, 0x73, 0x74, 0x52, 0x65, 0x63, + 0x65, 0x6e, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x6d, 0x70, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4c, 0x0a, 0x12, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x11, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x54, 0x0a, 0x0e, 0x72, 0x6f, 0x6f, 0x74, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x0d, 0x72, 0x6f, 0x6f, 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x2e, 0x0a, 0x11, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x5f, 0x62, + 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x13, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x61, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x30, 0x0a, 0x12, 0x69, 0x6e, 0x68, 0x65, 0x72, 0x69, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x75, + 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x14, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x69, 0x6e, 0x68, + 0x65, 0x72, 0x69, 0x74, 0x65, 0x64, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x24, 0x0a, 0x0c, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x15, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x69, 0x72, 0x73, 0x74, 0x52, 0x75, 0x6e, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x22, 0xbe, 0x03, 0x0a, 0x17, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x47, + 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, + 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x5b, 0x0a, 0x1a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x18, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, 0x14, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x54, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x60, 0x0a, 0x1d, 0x64, 0x65, 0x66, 0x61, + 0x75, 0x6c, 0x74, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x1a, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x0d, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x6d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, + 0x2e, 0x55, 0x73, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x0c, 0x75, 0x73, + 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x22, 0xc2, 0x08, + 0x0a, 0x13, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, + 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x23, 0x0a, 0x0b, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, + 0x74, 0x79, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x61, 0x63, 0x74, 0x69, 0x76, + 0x69, 0x74, 0x79, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x79, 0x70, + 0x65, 0x52, 0x0c, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x45, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x2b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x41, + 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x51, 0x0a, 0x11, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, + 0x61, 0x74, 0x5f, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, + 0x10, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x4e, 0x0a, 0x13, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x68, 0x65, 0x61, + 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x11, 0x6c, 0x61, 0x73, 0x74, + 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x4a, 0x0a, 0x11, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, + 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x65, + 0x6d, 0x70, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, + 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x5f, + 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x6d, + 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x45, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x54, 0x69, 0x6d, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0f, 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0e, 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0c, 0x6c, 0x61, 0x73, + 0x74, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, + 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, + 0x0b, 0x6c, 0x61, 0x73, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x34, 0x0a, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x6c, 0x61, 0x73, + 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x4f, 0x0a, 0x15, 0x75, 0x73, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x48, 0x00, 0x52, 0x12, 0x75, 0x73, 0x65, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x54, 0x0a, 0x24, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, + 0x6e, 0x74, 0x6c, 0x79, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x5f, 0x62, 0x75, 0x69, + 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x20, 0x6c, 0x61, + 0x73, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x41, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x69, 0x0a, 0x19, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x0f, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, + 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x16, 0x6c, 0x61, 0x73, + 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x6d, 0x70, 0x42, 0x02, 0x68, 0x00, 0x42, 0x13, 0x0a, 0x11, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, + 0x64, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x22, 0x92, 0x02, 0x0a, 0x19, 0x50, 0x65, + 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x19, 0x0a, 0x06, 0x72, 0x75, 0x6e, + 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x72, 0x75, 0x6e, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x30, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x69, + 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x74, 0x65, + 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x13, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, + 0x5f, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, + 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x11, 0x70, 0x61, 0x72, 0x65, + 0x6e, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, + 0x22, 0xe4, 0x02, 0x0a, 0x17, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x49, 0x0a, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, + 0x2e, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, + 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x45, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x74, + 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x54, 0x69, 0x6d, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x56, 0x0a, 0x17, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x61, 0x6c, + 0x5f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, + 0x15, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x61, 0x6c, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, + 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, + 0x70, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, + 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0x53, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x65, 0x74, 0x50, 0x6f, 0x69, + 0x6e, 0x74, 0x73, 0x12, 0x44, 0x0a, 0x06, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x65, + 0x74, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x06, 0x70, 0x6f, 0x69, 0x6e, 0x74, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0xe9, 0x02, 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x50, 0x6f, + 0x69, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x0a, 0x08, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, + 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x2b, 0x0a, 0x0f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x63, + 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x62, 0x69, + 0x6e, 0x61, 0x72, 0x79, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x19, 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x72, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x20, 0x66, 0x69, 0x72, 0x73, + 0x74, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, + 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x1c, 0x66, 0x69, 0x72, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, + 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x3f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, + 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, + 0x0a, 0x0b, 0x65, 0x78, 0x70, 0x69, 0x72, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0a, 0x65, 0x78, + 0x70, 0x69, 0x72, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x22, 0x0a, 0x0a, 0x72, + 0x65, 0x73, 0x65, 0x74, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0a, 0x72, 0x65, 0x73, 0x65, 0x74, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8d, + 0x08, 0x0a, 0x18, 0x4e, 0x65, 0x77, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, + 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, + 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x09, + 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, + 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x05, 0x69, 0x6e, 0x70, + 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5b, 0x0a, 0x1a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x18, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, + 0x14, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x6f, 0x75, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, + 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x51, 0x0a, 0x15, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x13, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x54, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x69, 0x0a, 0x18, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x5f, 0x72, 0x65, 0x75, 0x73, 0x65, 0x5f, 0x70, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x52, 0x65, 0x75, 0x73, 0x65, 0x50, + 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x15, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, + 0x52, 0x65, 0x75, 0x73, 0x65, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, + 0x0a, 0x0c, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x09, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, + 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, + 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, 0x0a, 0x0d, 0x63, 0x72, 0x6f, 0x6e, 0x5f, 0x73, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x72, + 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, + 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x6d, 0x6f, 0x52, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x59, 0x0a, 0x11, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x52, 0x10, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x4a, 0x0a, 0x0d, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x4d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x0c, 0x75, 0x73, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x22, 0xe9, 0x05, 0x0a, 0x0c, 0x43, 0x61, 0x6c, 0x6c, + 0x62, 0x61, 0x63, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x40, 0x0a, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x62, + 0x61, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x62, + 0x61, 0x63, 0x6b, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4c, 0x0a, 0x07, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, + 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, + 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x54, 0x72, 0x69, + 0x67, 0x67, 0x65, 0x72, 0x52, 0x07, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x4b, 0x0a, 0x11, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x10, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x43, + 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x5b, 0x0a, 0x1a, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, + 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, + 0x17, 0x6c, 0x61, 0x73, 0x74, 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6c, + 0x65, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x56, 0x0a, 0x14, 0x6c, 0x61, + 0x73, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x41, 0x74, 0x74, + 0x65, 0x6d, 0x70, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5b, + 0x0a, 0x1a, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x5f, 0x73, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x17, 0x6e, 0x65, 0x78, 0x74, + 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x69, + 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x10, 0x0a, 0x0e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x64, 0x1a, 0x7a, 0x0a, 0x07, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, + 0x72, 0x12, 0x64, 0x0a, 0x0f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6c, + 0x6f, 0x73, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x2e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x64, 0x48, 0x00, 0x52, + 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x42, 0x09, 0x0a, 0x07, 0x76, 0x61, 0x72, 0x69, 0x61, 0x6e, 0x74, 0x22, 0xdd, 0x06, 0x0a, + 0x19, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1e, 0x0a, 0x08, 0x65, 0x6e, 0x64, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x58, 0x0a, 0x19, 0x73, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x16, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x6f, 0x43, + 0x6c, 0x6f, 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x45, + 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, + 0x0d, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x31, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x4e, 0x65, + 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x61, 0x74, + 0x74, 0x65, 0x6d, 0x70, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x61, 0x74, 0x74, 0x65, + 0x6d, 0x70, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5b, 0x0a, 0x1a, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x61, + 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x52, 0x17, 0x6c, 0x61, 0x73, 0x74, 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, + 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x56, 0x0a, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x5f, 0x66, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x12, 0x6c, 0x61, 0x73, + 0x74, 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x5b, 0x0a, 0x1a, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x6d, + 0x70, 0x74, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, + 0x17, 0x6e, 0x65, 0x78, 0x74, 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x53, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x69, 0x0a, 0x11, 0x63, 0x61, + 0x6e, 0x63, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x0c, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x4e, + 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, + 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x10, 0x63, 0x61, 0x6e, + 0x63, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x30, 0x0a, 0x12, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, + 0xe9, 0x03, 0x0a, 0x1e, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x45, 0x0a, 0x0e, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x52, 0x0d, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x50, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x36, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x6c, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5b, + 0x0a, 0x1a, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x5f, 0x63, 0x6f, + 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x17, 0x6c, 0x61, 0x73, 0x74, + 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x54, + 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x56, 0x0a, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x61, + 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, + 0x75, 0x72, 0x65, 0x52, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x46, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5b, 0x0a, 0x1a, 0x6e, 0x65, 0x78, + 0x74, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x17, 0x6e, 0x65, 0x78, 0x74, 0x41, 0x74, 0x74, 0x65, + 0x6d, 0x70, 0x74, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x42, 0x93, 0x01, 0x0a, 0x1b, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, + 0x42, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, + 0x27, 0x67, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, + 0x70, 0x69, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2f, 0x76, 0x31, 0x3b, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0xaa, 0x02, 0x1a, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, 0x69, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x56, 0x31, 0xea, 0x02, 0x1d, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, 0x3a, + 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x3a, 0x3a, 0x56, 0x31, + 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_workflow_v1_message_proto_rawDescOnce sync.Once + file_temporal_api_workflow_v1_message_proto_rawDescData = file_temporal_api_workflow_v1_message_proto_rawDesc +) + +func file_temporal_api_workflow_v1_message_proto_rawDescGZIP() []byte { + file_temporal_api_workflow_v1_message_proto_rawDescOnce.Do(func() { + file_temporal_api_workflow_v1_message_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_workflow_v1_message_proto_rawDescData) + }) + return file_temporal_api_workflow_v1_message_proto_rawDescData +} + +var file_temporal_api_workflow_v1_message_proto_msgTypes = make([]protoimpl.MessageInfo, 13) +var file_temporal_api_workflow_v1_message_proto_goTypes = []any{ + (*WorkflowExecutionInfo)(nil), // 0: temporal.api.workflow.v1.WorkflowExecutionInfo + (*WorkflowExecutionConfig)(nil), // 1: temporal.api.workflow.v1.WorkflowExecutionConfig + (*PendingActivityInfo)(nil), // 2: temporal.api.workflow.v1.PendingActivityInfo + (*PendingChildExecutionInfo)(nil), // 3: temporal.api.workflow.v1.PendingChildExecutionInfo + (*PendingWorkflowTaskInfo)(nil), // 4: temporal.api.workflow.v1.PendingWorkflowTaskInfo + (*ResetPoints)(nil), // 5: temporal.api.workflow.v1.ResetPoints + (*ResetPointInfo)(nil), // 6: temporal.api.workflow.v1.ResetPointInfo + (*NewWorkflowExecutionInfo)(nil), // 7: temporal.api.workflow.v1.NewWorkflowExecutionInfo + (*CallbackInfo)(nil), // 8: temporal.api.workflow.v1.CallbackInfo + (*PendingNexusOperationInfo)(nil), // 9: temporal.api.workflow.v1.PendingNexusOperationInfo + (*NexusOperationCancellationInfo)(nil), // 10: temporal.api.workflow.v1.NexusOperationCancellationInfo + (*CallbackInfo_WorkflowClosed)(nil), // 11: temporal.api.workflow.v1.CallbackInfo.WorkflowClosed + (*CallbackInfo_Trigger)(nil), // 12: temporal.api.workflow.v1.CallbackInfo.Trigger + (*v1.WorkflowExecution)(nil), // 13: temporal.api.common.v1.WorkflowExecution + (*v1.WorkflowType)(nil), // 14: temporal.api.common.v1.WorkflowType + (*timestamppb.Timestamp)(nil), // 15: google.protobuf.Timestamp + (v11.WorkflowExecutionStatus)(0), // 16: temporal.api.enums.v1.WorkflowExecutionStatus + (*v1.Memo)(nil), // 17: temporal.api.common.v1.Memo + (*v1.SearchAttributes)(nil), // 18: temporal.api.common.v1.SearchAttributes + (*v1.WorkerVersionStamp)(nil), // 19: temporal.api.common.v1.WorkerVersionStamp + (*durationpb.Duration)(nil), // 20: google.protobuf.Duration + (*v12.TaskQueue)(nil), // 21: temporal.api.taskqueue.v1.TaskQueue + (*v13.UserMetadata)(nil), // 22: temporal.api.sdk.v1.UserMetadata + (*v1.ActivityType)(nil), // 23: temporal.api.common.v1.ActivityType + (v11.PendingActivityState)(0), // 24: temporal.api.enums.v1.PendingActivityState + (*v1.Payloads)(nil), // 25: temporal.api.common.v1.Payloads + (*v14.Failure)(nil), // 26: temporal.api.failure.v1.Failure + (*emptypb.Empty)(nil), // 27: google.protobuf.Empty + (v11.ParentClosePolicy)(0), // 28: temporal.api.enums.v1.ParentClosePolicy + (v11.PendingWorkflowTaskState)(0), // 29: temporal.api.enums.v1.PendingWorkflowTaskState + (v11.WorkflowIdReusePolicy)(0), // 30: temporal.api.enums.v1.WorkflowIdReusePolicy + (*v1.RetryPolicy)(nil), // 31: temporal.api.common.v1.RetryPolicy + (*v1.Header)(nil), // 32: temporal.api.common.v1.Header + (*v1.Callback)(nil), // 33: temporal.api.common.v1.Callback + (v11.CallbackState)(0), // 34: temporal.api.enums.v1.CallbackState + (v11.PendingNexusOperationState)(0), // 35: temporal.api.enums.v1.PendingNexusOperationState + (v11.NexusOperationCancellationState)(0), // 36: temporal.api.enums.v1.NexusOperationCancellationState +} +var file_temporal_api_workflow_v1_message_proto_depIdxs = []int32{ + 13, // 0: temporal.api.workflow.v1.WorkflowExecutionInfo.execution:type_name -> temporal.api.common.v1.WorkflowExecution + 14, // 1: temporal.api.workflow.v1.WorkflowExecutionInfo.type:type_name -> temporal.api.common.v1.WorkflowType + 15, // 2: temporal.api.workflow.v1.WorkflowExecutionInfo.start_time:type_name -> google.protobuf.Timestamp + 15, // 3: temporal.api.workflow.v1.WorkflowExecutionInfo.close_time:type_name -> google.protobuf.Timestamp + 16, // 4: temporal.api.workflow.v1.WorkflowExecutionInfo.status:type_name -> temporal.api.enums.v1.WorkflowExecutionStatus + 13, // 5: temporal.api.workflow.v1.WorkflowExecutionInfo.parent_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 15, // 6: temporal.api.workflow.v1.WorkflowExecutionInfo.execution_time:type_name -> google.protobuf.Timestamp + 17, // 7: temporal.api.workflow.v1.WorkflowExecutionInfo.memo:type_name -> temporal.api.common.v1.Memo + 18, // 8: temporal.api.workflow.v1.WorkflowExecutionInfo.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 5, // 9: temporal.api.workflow.v1.WorkflowExecutionInfo.auto_reset_points:type_name -> temporal.api.workflow.v1.ResetPoints + 19, // 10: temporal.api.workflow.v1.WorkflowExecutionInfo.most_recent_worker_version_stamp:type_name -> temporal.api.common.v1.WorkerVersionStamp + 20, // 11: temporal.api.workflow.v1.WorkflowExecutionInfo.execution_duration:type_name -> google.protobuf.Duration + 13, // 12: temporal.api.workflow.v1.WorkflowExecutionInfo.root_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 21, // 13: temporal.api.workflow.v1.WorkflowExecutionConfig.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 20, // 14: temporal.api.workflow.v1.WorkflowExecutionConfig.workflow_execution_timeout:type_name -> google.protobuf.Duration + 20, // 15: temporal.api.workflow.v1.WorkflowExecutionConfig.workflow_run_timeout:type_name -> google.protobuf.Duration + 20, // 16: temporal.api.workflow.v1.WorkflowExecutionConfig.default_workflow_task_timeout:type_name -> google.protobuf.Duration + 22, // 17: temporal.api.workflow.v1.WorkflowExecutionConfig.user_metadata:type_name -> temporal.api.sdk.v1.UserMetadata + 23, // 18: temporal.api.workflow.v1.PendingActivityInfo.activity_type:type_name -> temporal.api.common.v1.ActivityType + 24, // 19: temporal.api.workflow.v1.PendingActivityInfo.state:type_name -> temporal.api.enums.v1.PendingActivityState + 25, // 20: temporal.api.workflow.v1.PendingActivityInfo.heartbeat_details:type_name -> temporal.api.common.v1.Payloads + 15, // 21: temporal.api.workflow.v1.PendingActivityInfo.last_heartbeat_time:type_name -> google.protobuf.Timestamp + 15, // 22: temporal.api.workflow.v1.PendingActivityInfo.last_started_time:type_name -> google.protobuf.Timestamp + 15, // 23: temporal.api.workflow.v1.PendingActivityInfo.scheduled_time:type_name -> google.protobuf.Timestamp + 15, // 24: temporal.api.workflow.v1.PendingActivityInfo.expiration_time:type_name -> google.protobuf.Timestamp + 26, // 25: temporal.api.workflow.v1.PendingActivityInfo.last_failure:type_name -> temporal.api.failure.v1.Failure + 27, // 26: temporal.api.workflow.v1.PendingActivityInfo.use_workflow_build_id:type_name -> google.protobuf.Empty + 19, // 27: temporal.api.workflow.v1.PendingActivityInfo.last_worker_version_stamp:type_name -> temporal.api.common.v1.WorkerVersionStamp + 28, // 28: temporal.api.workflow.v1.PendingChildExecutionInfo.parent_close_policy:type_name -> temporal.api.enums.v1.ParentClosePolicy + 29, // 29: temporal.api.workflow.v1.PendingWorkflowTaskInfo.state:type_name -> temporal.api.enums.v1.PendingWorkflowTaskState + 15, // 30: temporal.api.workflow.v1.PendingWorkflowTaskInfo.scheduled_time:type_name -> google.protobuf.Timestamp + 15, // 31: temporal.api.workflow.v1.PendingWorkflowTaskInfo.original_scheduled_time:type_name -> google.protobuf.Timestamp + 15, // 32: temporal.api.workflow.v1.PendingWorkflowTaskInfo.started_time:type_name -> google.protobuf.Timestamp + 6, // 33: temporal.api.workflow.v1.ResetPoints.points:type_name -> temporal.api.workflow.v1.ResetPointInfo + 15, // 34: temporal.api.workflow.v1.ResetPointInfo.create_time:type_name -> google.protobuf.Timestamp + 15, // 35: temporal.api.workflow.v1.ResetPointInfo.expire_time:type_name -> google.protobuf.Timestamp + 14, // 36: temporal.api.workflow.v1.NewWorkflowExecutionInfo.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 21, // 37: temporal.api.workflow.v1.NewWorkflowExecutionInfo.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 25, // 38: temporal.api.workflow.v1.NewWorkflowExecutionInfo.input:type_name -> temporal.api.common.v1.Payloads + 20, // 39: temporal.api.workflow.v1.NewWorkflowExecutionInfo.workflow_execution_timeout:type_name -> google.protobuf.Duration + 20, // 40: temporal.api.workflow.v1.NewWorkflowExecutionInfo.workflow_run_timeout:type_name -> google.protobuf.Duration + 20, // 41: temporal.api.workflow.v1.NewWorkflowExecutionInfo.workflow_task_timeout:type_name -> google.protobuf.Duration + 30, // 42: temporal.api.workflow.v1.NewWorkflowExecutionInfo.workflow_id_reuse_policy:type_name -> temporal.api.enums.v1.WorkflowIdReusePolicy + 31, // 43: temporal.api.workflow.v1.NewWorkflowExecutionInfo.retry_policy:type_name -> temporal.api.common.v1.RetryPolicy + 17, // 44: temporal.api.workflow.v1.NewWorkflowExecutionInfo.memo:type_name -> temporal.api.common.v1.Memo + 18, // 45: temporal.api.workflow.v1.NewWorkflowExecutionInfo.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 32, // 46: temporal.api.workflow.v1.NewWorkflowExecutionInfo.header:type_name -> temporal.api.common.v1.Header + 22, // 47: temporal.api.workflow.v1.NewWorkflowExecutionInfo.user_metadata:type_name -> temporal.api.sdk.v1.UserMetadata + 33, // 48: temporal.api.workflow.v1.CallbackInfo.callback:type_name -> temporal.api.common.v1.Callback + 12, // 49: temporal.api.workflow.v1.CallbackInfo.trigger:type_name -> temporal.api.workflow.v1.CallbackInfo.Trigger + 15, // 50: temporal.api.workflow.v1.CallbackInfo.registration_time:type_name -> google.protobuf.Timestamp + 34, // 51: temporal.api.workflow.v1.CallbackInfo.state:type_name -> temporal.api.enums.v1.CallbackState + 15, // 52: temporal.api.workflow.v1.CallbackInfo.last_attempt_complete_time:type_name -> google.protobuf.Timestamp + 26, // 53: temporal.api.workflow.v1.CallbackInfo.last_attempt_failure:type_name -> temporal.api.failure.v1.Failure + 15, // 54: temporal.api.workflow.v1.CallbackInfo.next_attempt_schedule_time:type_name -> google.protobuf.Timestamp + 20, // 55: temporal.api.workflow.v1.PendingNexusOperationInfo.schedule_to_close_timeout:type_name -> google.protobuf.Duration + 15, // 56: temporal.api.workflow.v1.PendingNexusOperationInfo.scheduled_time:type_name -> google.protobuf.Timestamp + 35, // 57: temporal.api.workflow.v1.PendingNexusOperationInfo.state:type_name -> temporal.api.enums.v1.PendingNexusOperationState + 15, // 58: temporal.api.workflow.v1.PendingNexusOperationInfo.last_attempt_complete_time:type_name -> google.protobuf.Timestamp + 26, // 59: temporal.api.workflow.v1.PendingNexusOperationInfo.last_attempt_failure:type_name -> temporal.api.failure.v1.Failure + 15, // 60: temporal.api.workflow.v1.PendingNexusOperationInfo.next_attempt_schedule_time:type_name -> google.protobuf.Timestamp + 10, // 61: temporal.api.workflow.v1.PendingNexusOperationInfo.cancellation_info:type_name -> temporal.api.workflow.v1.NexusOperationCancellationInfo + 15, // 62: temporal.api.workflow.v1.NexusOperationCancellationInfo.requested_time:type_name -> google.protobuf.Timestamp + 36, // 63: temporal.api.workflow.v1.NexusOperationCancellationInfo.state:type_name -> temporal.api.enums.v1.NexusOperationCancellationState + 15, // 64: temporal.api.workflow.v1.NexusOperationCancellationInfo.last_attempt_complete_time:type_name -> google.protobuf.Timestamp + 26, // 65: temporal.api.workflow.v1.NexusOperationCancellationInfo.last_attempt_failure:type_name -> temporal.api.failure.v1.Failure + 15, // 66: temporal.api.workflow.v1.NexusOperationCancellationInfo.next_attempt_schedule_time:type_name -> google.protobuf.Timestamp + 11, // 67: temporal.api.workflow.v1.CallbackInfo.Trigger.workflow_closed:type_name -> temporal.api.workflow.v1.CallbackInfo.WorkflowClosed + 68, // [68:68] is the sub-list for method output_type + 68, // [68:68] is the sub-list for method input_type + 68, // [68:68] is the sub-list for extension type_name + 68, // [68:68] is the sub-list for extension extendee + 0, // [0:68] is the sub-list for field type_name +} + +func init() { file_temporal_api_workflow_v1_message_proto_init() } +func file_temporal_api_workflow_v1_message_proto_init() { + if File_temporal_api_workflow_v1_message_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_workflow_v1_message_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflow_v1_message_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*WorkflowExecutionConfig); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflow_v1_message_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*PendingActivityInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflow_v1_message_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*PendingChildExecutionInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflow_v1_message_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*PendingWorkflowTaskInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflow_v1_message_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*ResetPoints); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflow_v1_message_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*ResetPointInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflow_v1_message_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*NewWorkflowExecutionInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflow_v1_message_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*CallbackInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflow_v1_message_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*PendingNexusOperationInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflow_v1_message_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*NexusOperationCancellationInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflow_v1_message_proto_msgTypes[11].Exporter = func(v any, i int) any { + switch v := v.(*CallbackInfo_WorkflowClosed); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflow_v1_message_proto_msgTypes[12].Exporter = func(v any, i int) any { + switch v := v.(*CallbackInfo_Trigger); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_temporal_api_workflow_v1_message_proto_msgTypes[2].OneofWrappers = []any{ + (*PendingActivityInfo_UseWorkflowBuildId)(nil), + (*PendingActivityInfo_LastIndependentlyAssignedBuildId)(nil), + } + file_temporal_api_workflow_v1_message_proto_msgTypes[12].OneofWrappers = []any{ + (*CallbackInfo_Trigger_WorkflowClosed)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_workflow_v1_message_proto_rawDesc, + NumEnums: 0, + NumMessages: 13, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_workflow_v1_message_proto_goTypes, + DependencyIndexes: file_temporal_api_workflow_v1_message_proto_depIdxs, + MessageInfos: file_temporal_api_workflow_v1_message_proto_msgTypes, + }.Build() + File_temporal_api_workflow_v1_message_proto = out.File + file_temporal_api_workflow_v1_message_proto_rawDesc = nil + file_temporal_api_workflow_v1_message_proto_goTypes = nil + file_temporal_api_workflow_v1_message_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/workflowservice/v1/request_response.go-helpers.pb.go b/vendor/go.temporal.io/api/workflowservice/v1/request_response.go-helpers.pb.go new file mode 100644 index 00000000000..aeaf8376d57 --- /dev/null +++ b/vendor/go.temporal.io/api/workflowservice/v1/request_response.go-helpers.pb.go @@ -0,0 +1,4690 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-helpers. DO NOT EDIT. +package workflowservice + +import ( + "google.golang.org/protobuf/proto" +) + +// Marshal an object of type RegisterNamespaceRequest to the protobuf v3 wire format +func (val *RegisterNamespaceRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RegisterNamespaceRequest from the protobuf v3 wire format +func (val *RegisterNamespaceRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RegisterNamespaceRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RegisterNamespaceRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RegisterNamespaceRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RegisterNamespaceRequest + switch t := that.(type) { + case *RegisterNamespaceRequest: + that1 = t + case RegisterNamespaceRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RegisterNamespaceResponse to the protobuf v3 wire format +func (val *RegisterNamespaceResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RegisterNamespaceResponse from the protobuf v3 wire format +func (val *RegisterNamespaceResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RegisterNamespaceResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RegisterNamespaceResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RegisterNamespaceResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RegisterNamespaceResponse + switch t := that.(type) { + case *RegisterNamespaceResponse: + that1 = t + case RegisterNamespaceResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListNamespacesRequest to the protobuf v3 wire format +func (val *ListNamespacesRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListNamespacesRequest from the protobuf v3 wire format +func (val *ListNamespacesRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListNamespacesRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListNamespacesRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListNamespacesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListNamespacesRequest + switch t := that.(type) { + case *ListNamespacesRequest: + that1 = t + case ListNamespacesRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListNamespacesResponse to the protobuf v3 wire format +func (val *ListNamespacesResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListNamespacesResponse from the protobuf v3 wire format +func (val *ListNamespacesResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListNamespacesResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListNamespacesResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListNamespacesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListNamespacesResponse + switch t := that.(type) { + case *ListNamespacesResponse: + that1 = t + case ListNamespacesResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DescribeNamespaceRequest to the protobuf v3 wire format +func (val *DescribeNamespaceRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DescribeNamespaceRequest from the protobuf v3 wire format +func (val *DescribeNamespaceRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DescribeNamespaceRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DescribeNamespaceRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DescribeNamespaceRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DescribeNamespaceRequest + switch t := that.(type) { + case *DescribeNamespaceRequest: + that1 = t + case DescribeNamespaceRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DescribeNamespaceResponse to the protobuf v3 wire format +func (val *DescribeNamespaceResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DescribeNamespaceResponse from the protobuf v3 wire format +func (val *DescribeNamespaceResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DescribeNamespaceResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DescribeNamespaceResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DescribeNamespaceResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DescribeNamespaceResponse + switch t := that.(type) { + case *DescribeNamespaceResponse: + that1 = t + case DescribeNamespaceResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateNamespaceRequest to the protobuf v3 wire format +func (val *UpdateNamespaceRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateNamespaceRequest from the protobuf v3 wire format +func (val *UpdateNamespaceRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateNamespaceRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateNamespaceRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateNamespaceRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateNamespaceRequest + switch t := that.(type) { + case *UpdateNamespaceRequest: + that1 = t + case UpdateNamespaceRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateNamespaceResponse to the protobuf v3 wire format +func (val *UpdateNamespaceResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateNamespaceResponse from the protobuf v3 wire format +func (val *UpdateNamespaceResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateNamespaceResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateNamespaceResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateNamespaceResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateNamespaceResponse + switch t := that.(type) { + case *UpdateNamespaceResponse: + that1 = t + case UpdateNamespaceResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeprecateNamespaceRequest to the protobuf v3 wire format +func (val *DeprecateNamespaceRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeprecateNamespaceRequest from the protobuf v3 wire format +func (val *DeprecateNamespaceRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeprecateNamespaceRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeprecateNamespaceRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeprecateNamespaceRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeprecateNamespaceRequest + switch t := that.(type) { + case *DeprecateNamespaceRequest: + that1 = t + case DeprecateNamespaceRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeprecateNamespaceResponse to the protobuf v3 wire format +func (val *DeprecateNamespaceResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeprecateNamespaceResponse from the protobuf v3 wire format +func (val *DeprecateNamespaceResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeprecateNamespaceResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeprecateNamespaceResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeprecateNamespaceResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeprecateNamespaceResponse + switch t := that.(type) { + case *DeprecateNamespaceResponse: + that1 = t + case DeprecateNamespaceResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StartWorkflowExecutionRequest to the protobuf v3 wire format +func (val *StartWorkflowExecutionRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StartWorkflowExecutionRequest from the protobuf v3 wire format +func (val *StartWorkflowExecutionRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StartWorkflowExecutionRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StartWorkflowExecutionRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StartWorkflowExecutionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StartWorkflowExecutionRequest + switch t := that.(type) { + case *StartWorkflowExecutionRequest: + that1 = t + case StartWorkflowExecutionRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StartWorkflowExecutionResponse to the protobuf v3 wire format +func (val *StartWorkflowExecutionResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StartWorkflowExecutionResponse from the protobuf v3 wire format +func (val *StartWorkflowExecutionResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StartWorkflowExecutionResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StartWorkflowExecutionResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StartWorkflowExecutionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StartWorkflowExecutionResponse + switch t := that.(type) { + case *StartWorkflowExecutionResponse: + that1 = t + case StartWorkflowExecutionResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetWorkflowExecutionHistoryRequest to the protobuf v3 wire format +func (val *GetWorkflowExecutionHistoryRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetWorkflowExecutionHistoryRequest from the protobuf v3 wire format +func (val *GetWorkflowExecutionHistoryRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetWorkflowExecutionHistoryRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetWorkflowExecutionHistoryRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetWorkflowExecutionHistoryRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetWorkflowExecutionHistoryRequest + switch t := that.(type) { + case *GetWorkflowExecutionHistoryRequest: + that1 = t + case GetWorkflowExecutionHistoryRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetWorkflowExecutionHistoryResponse to the protobuf v3 wire format +func (val *GetWorkflowExecutionHistoryResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetWorkflowExecutionHistoryResponse from the protobuf v3 wire format +func (val *GetWorkflowExecutionHistoryResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetWorkflowExecutionHistoryResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetWorkflowExecutionHistoryResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetWorkflowExecutionHistoryResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetWorkflowExecutionHistoryResponse + switch t := that.(type) { + case *GetWorkflowExecutionHistoryResponse: + that1 = t + case GetWorkflowExecutionHistoryResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetWorkflowExecutionHistoryReverseRequest to the protobuf v3 wire format +func (val *GetWorkflowExecutionHistoryReverseRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetWorkflowExecutionHistoryReverseRequest from the protobuf v3 wire format +func (val *GetWorkflowExecutionHistoryReverseRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetWorkflowExecutionHistoryReverseRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetWorkflowExecutionHistoryReverseRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetWorkflowExecutionHistoryReverseRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetWorkflowExecutionHistoryReverseRequest + switch t := that.(type) { + case *GetWorkflowExecutionHistoryReverseRequest: + that1 = t + case GetWorkflowExecutionHistoryReverseRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetWorkflowExecutionHistoryReverseResponse to the protobuf v3 wire format +func (val *GetWorkflowExecutionHistoryReverseResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetWorkflowExecutionHistoryReverseResponse from the protobuf v3 wire format +func (val *GetWorkflowExecutionHistoryReverseResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetWorkflowExecutionHistoryReverseResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetWorkflowExecutionHistoryReverseResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetWorkflowExecutionHistoryReverseResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetWorkflowExecutionHistoryReverseResponse + switch t := that.(type) { + case *GetWorkflowExecutionHistoryReverseResponse: + that1 = t + case GetWorkflowExecutionHistoryReverseResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PollWorkflowTaskQueueRequest to the protobuf v3 wire format +func (val *PollWorkflowTaskQueueRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PollWorkflowTaskQueueRequest from the protobuf v3 wire format +func (val *PollWorkflowTaskQueueRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PollWorkflowTaskQueueRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PollWorkflowTaskQueueRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PollWorkflowTaskQueueRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PollWorkflowTaskQueueRequest + switch t := that.(type) { + case *PollWorkflowTaskQueueRequest: + that1 = t + case PollWorkflowTaskQueueRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PollWorkflowTaskQueueResponse to the protobuf v3 wire format +func (val *PollWorkflowTaskQueueResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PollWorkflowTaskQueueResponse from the protobuf v3 wire format +func (val *PollWorkflowTaskQueueResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PollWorkflowTaskQueueResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PollWorkflowTaskQueueResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PollWorkflowTaskQueueResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PollWorkflowTaskQueueResponse + switch t := that.(type) { + case *PollWorkflowTaskQueueResponse: + that1 = t + case PollWorkflowTaskQueueResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondWorkflowTaskCompletedRequest to the protobuf v3 wire format +func (val *RespondWorkflowTaskCompletedRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondWorkflowTaskCompletedRequest from the protobuf v3 wire format +func (val *RespondWorkflowTaskCompletedRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondWorkflowTaskCompletedRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondWorkflowTaskCompletedRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondWorkflowTaskCompletedRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondWorkflowTaskCompletedRequest + switch t := that.(type) { + case *RespondWorkflowTaskCompletedRequest: + that1 = t + case RespondWorkflowTaskCompletedRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondWorkflowTaskCompletedResponse to the protobuf v3 wire format +func (val *RespondWorkflowTaskCompletedResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondWorkflowTaskCompletedResponse from the protobuf v3 wire format +func (val *RespondWorkflowTaskCompletedResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondWorkflowTaskCompletedResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondWorkflowTaskCompletedResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondWorkflowTaskCompletedResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondWorkflowTaskCompletedResponse + switch t := that.(type) { + case *RespondWorkflowTaskCompletedResponse: + that1 = t + case RespondWorkflowTaskCompletedResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondWorkflowTaskFailedRequest to the protobuf v3 wire format +func (val *RespondWorkflowTaskFailedRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondWorkflowTaskFailedRequest from the protobuf v3 wire format +func (val *RespondWorkflowTaskFailedRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondWorkflowTaskFailedRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondWorkflowTaskFailedRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondWorkflowTaskFailedRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondWorkflowTaskFailedRequest + switch t := that.(type) { + case *RespondWorkflowTaskFailedRequest: + that1 = t + case RespondWorkflowTaskFailedRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondWorkflowTaskFailedResponse to the protobuf v3 wire format +func (val *RespondWorkflowTaskFailedResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondWorkflowTaskFailedResponse from the protobuf v3 wire format +func (val *RespondWorkflowTaskFailedResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondWorkflowTaskFailedResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondWorkflowTaskFailedResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondWorkflowTaskFailedResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondWorkflowTaskFailedResponse + switch t := that.(type) { + case *RespondWorkflowTaskFailedResponse: + that1 = t + case RespondWorkflowTaskFailedResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PollActivityTaskQueueRequest to the protobuf v3 wire format +func (val *PollActivityTaskQueueRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PollActivityTaskQueueRequest from the protobuf v3 wire format +func (val *PollActivityTaskQueueRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PollActivityTaskQueueRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PollActivityTaskQueueRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PollActivityTaskQueueRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PollActivityTaskQueueRequest + switch t := that.(type) { + case *PollActivityTaskQueueRequest: + that1 = t + case PollActivityTaskQueueRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PollActivityTaskQueueResponse to the protobuf v3 wire format +func (val *PollActivityTaskQueueResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PollActivityTaskQueueResponse from the protobuf v3 wire format +func (val *PollActivityTaskQueueResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PollActivityTaskQueueResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PollActivityTaskQueueResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PollActivityTaskQueueResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PollActivityTaskQueueResponse + switch t := that.(type) { + case *PollActivityTaskQueueResponse: + that1 = t + case PollActivityTaskQueueResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RecordActivityTaskHeartbeatRequest to the protobuf v3 wire format +func (val *RecordActivityTaskHeartbeatRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RecordActivityTaskHeartbeatRequest from the protobuf v3 wire format +func (val *RecordActivityTaskHeartbeatRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RecordActivityTaskHeartbeatRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RecordActivityTaskHeartbeatRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RecordActivityTaskHeartbeatRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RecordActivityTaskHeartbeatRequest + switch t := that.(type) { + case *RecordActivityTaskHeartbeatRequest: + that1 = t + case RecordActivityTaskHeartbeatRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RecordActivityTaskHeartbeatResponse to the protobuf v3 wire format +func (val *RecordActivityTaskHeartbeatResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RecordActivityTaskHeartbeatResponse from the protobuf v3 wire format +func (val *RecordActivityTaskHeartbeatResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RecordActivityTaskHeartbeatResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RecordActivityTaskHeartbeatResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RecordActivityTaskHeartbeatResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RecordActivityTaskHeartbeatResponse + switch t := that.(type) { + case *RecordActivityTaskHeartbeatResponse: + that1 = t + case RecordActivityTaskHeartbeatResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RecordActivityTaskHeartbeatByIdRequest to the protobuf v3 wire format +func (val *RecordActivityTaskHeartbeatByIdRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RecordActivityTaskHeartbeatByIdRequest from the protobuf v3 wire format +func (val *RecordActivityTaskHeartbeatByIdRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RecordActivityTaskHeartbeatByIdRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RecordActivityTaskHeartbeatByIdRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RecordActivityTaskHeartbeatByIdRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RecordActivityTaskHeartbeatByIdRequest + switch t := that.(type) { + case *RecordActivityTaskHeartbeatByIdRequest: + that1 = t + case RecordActivityTaskHeartbeatByIdRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RecordActivityTaskHeartbeatByIdResponse to the protobuf v3 wire format +func (val *RecordActivityTaskHeartbeatByIdResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RecordActivityTaskHeartbeatByIdResponse from the protobuf v3 wire format +func (val *RecordActivityTaskHeartbeatByIdResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RecordActivityTaskHeartbeatByIdResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RecordActivityTaskHeartbeatByIdResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RecordActivityTaskHeartbeatByIdResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RecordActivityTaskHeartbeatByIdResponse + switch t := that.(type) { + case *RecordActivityTaskHeartbeatByIdResponse: + that1 = t + case RecordActivityTaskHeartbeatByIdResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondActivityTaskCompletedRequest to the protobuf v3 wire format +func (val *RespondActivityTaskCompletedRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondActivityTaskCompletedRequest from the protobuf v3 wire format +func (val *RespondActivityTaskCompletedRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondActivityTaskCompletedRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondActivityTaskCompletedRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondActivityTaskCompletedRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondActivityTaskCompletedRequest + switch t := that.(type) { + case *RespondActivityTaskCompletedRequest: + that1 = t + case RespondActivityTaskCompletedRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondActivityTaskCompletedResponse to the protobuf v3 wire format +func (val *RespondActivityTaskCompletedResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondActivityTaskCompletedResponse from the protobuf v3 wire format +func (val *RespondActivityTaskCompletedResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondActivityTaskCompletedResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondActivityTaskCompletedResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondActivityTaskCompletedResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondActivityTaskCompletedResponse + switch t := that.(type) { + case *RespondActivityTaskCompletedResponse: + that1 = t + case RespondActivityTaskCompletedResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondActivityTaskCompletedByIdRequest to the protobuf v3 wire format +func (val *RespondActivityTaskCompletedByIdRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondActivityTaskCompletedByIdRequest from the protobuf v3 wire format +func (val *RespondActivityTaskCompletedByIdRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondActivityTaskCompletedByIdRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondActivityTaskCompletedByIdRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondActivityTaskCompletedByIdRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondActivityTaskCompletedByIdRequest + switch t := that.(type) { + case *RespondActivityTaskCompletedByIdRequest: + that1 = t + case RespondActivityTaskCompletedByIdRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondActivityTaskCompletedByIdResponse to the protobuf v3 wire format +func (val *RespondActivityTaskCompletedByIdResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondActivityTaskCompletedByIdResponse from the protobuf v3 wire format +func (val *RespondActivityTaskCompletedByIdResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondActivityTaskCompletedByIdResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondActivityTaskCompletedByIdResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondActivityTaskCompletedByIdResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondActivityTaskCompletedByIdResponse + switch t := that.(type) { + case *RespondActivityTaskCompletedByIdResponse: + that1 = t + case RespondActivityTaskCompletedByIdResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondActivityTaskFailedRequest to the protobuf v3 wire format +func (val *RespondActivityTaskFailedRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondActivityTaskFailedRequest from the protobuf v3 wire format +func (val *RespondActivityTaskFailedRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondActivityTaskFailedRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondActivityTaskFailedRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondActivityTaskFailedRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondActivityTaskFailedRequest + switch t := that.(type) { + case *RespondActivityTaskFailedRequest: + that1 = t + case RespondActivityTaskFailedRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondActivityTaskFailedResponse to the protobuf v3 wire format +func (val *RespondActivityTaskFailedResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondActivityTaskFailedResponse from the protobuf v3 wire format +func (val *RespondActivityTaskFailedResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondActivityTaskFailedResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondActivityTaskFailedResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondActivityTaskFailedResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondActivityTaskFailedResponse + switch t := that.(type) { + case *RespondActivityTaskFailedResponse: + that1 = t + case RespondActivityTaskFailedResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondActivityTaskFailedByIdRequest to the protobuf v3 wire format +func (val *RespondActivityTaskFailedByIdRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondActivityTaskFailedByIdRequest from the protobuf v3 wire format +func (val *RespondActivityTaskFailedByIdRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondActivityTaskFailedByIdRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondActivityTaskFailedByIdRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondActivityTaskFailedByIdRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondActivityTaskFailedByIdRequest + switch t := that.(type) { + case *RespondActivityTaskFailedByIdRequest: + that1 = t + case RespondActivityTaskFailedByIdRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondActivityTaskFailedByIdResponse to the protobuf v3 wire format +func (val *RespondActivityTaskFailedByIdResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondActivityTaskFailedByIdResponse from the protobuf v3 wire format +func (val *RespondActivityTaskFailedByIdResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondActivityTaskFailedByIdResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondActivityTaskFailedByIdResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondActivityTaskFailedByIdResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondActivityTaskFailedByIdResponse + switch t := that.(type) { + case *RespondActivityTaskFailedByIdResponse: + that1 = t + case RespondActivityTaskFailedByIdResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondActivityTaskCanceledRequest to the protobuf v3 wire format +func (val *RespondActivityTaskCanceledRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondActivityTaskCanceledRequest from the protobuf v3 wire format +func (val *RespondActivityTaskCanceledRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondActivityTaskCanceledRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondActivityTaskCanceledRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondActivityTaskCanceledRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondActivityTaskCanceledRequest + switch t := that.(type) { + case *RespondActivityTaskCanceledRequest: + that1 = t + case RespondActivityTaskCanceledRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondActivityTaskCanceledResponse to the protobuf v3 wire format +func (val *RespondActivityTaskCanceledResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondActivityTaskCanceledResponse from the protobuf v3 wire format +func (val *RespondActivityTaskCanceledResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondActivityTaskCanceledResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondActivityTaskCanceledResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondActivityTaskCanceledResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondActivityTaskCanceledResponse + switch t := that.(type) { + case *RespondActivityTaskCanceledResponse: + that1 = t + case RespondActivityTaskCanceledResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondActivityTaskCanceledByIdRequest to the protobuf v3 wire format +func (val *RespondActivityTaskCanceledByIdRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondActivityTaskCanceledByIdRequest from the protobuf v3 wire format +func (val *RespondActivityTaskCanceledByIdRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondActivityTaskCanceledByIdRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondActivityTaskCanceledByIdRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondActivityTaskCanceledByIdRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondActivityTaskCanceledByIdRequest + switch t := that.(type) { + case *RespondActivityTaskCanceledByIdRequest: + that1 = t + case RespondActivityTaskCanceledByIdRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondActivityTaskCanceledByIdResponse to the protobuf v3 wire format +func (val *RespondActivityTaskCanceledByIdResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondActivityTaskCanceledByIdResponse from the protobuf v3 wire format +func (val *RespondActivityTaskCanceledByIdResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondActivityTaskCanceledByIdResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondActivityTaskCanceledByIdResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondActivityTaskCanceledByIdResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondActivityTaskCanceledByIdResponse + switch t := that.(type) { + case *RespondActivityTaskCanceledByIdResponse: + that1 = t + case RespondActivityTaskCanceledByIdResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RequestCancelWorkflowExecutionRequest to the protobuf v3 wire format +func (val *RequestCancelWorkflowExecutionRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RequestCancelWorkflowExecutionRequest from the protobuf v3 wire format +func (val *RequestCancelWorkflowExecutionRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RequestCancelWorkflowExecutionRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RequestCancelWorkflowExecutionRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RequestCancelWorkflowExecutionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RequestCancelWorkflowExecutionRequest + switch t := that.(type) { + case *RequestCancelWorkflowExecutionRequest: + that1 = t + case RequestCancelWorkflowExecutionRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RequestCancelWorkflowExecutionResponse to the protobuf v3 wire format +func (val *RequestCancelWorkflowExecutionResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RequestCancelWorkflowExecutionResponse from the protobuf v3 wire format +func (val *RequestCancelWorkflowExecutionResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RequestCancelWorkflowExecutionResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RequestCancelWorkflowExecutionResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RequestCancelWorkflowExecutionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RequestCancelWorkflowExecutionResponse + switch t := that.(type) { + case *RequestCancelWorkflowExecutionResponse: + that1 = t + case RequestCancelWorkflowExecutionResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SignalWorkflowExecutionRequest to the protobuf v3 wire format +func (val *SignalWorkflowExecutionRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SignalWorkflowExecutionRequest from the protobuf v3 wire format +func (val *SignalWorkflowExecutionRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SignalWorkflowExecutionRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SignalWorkflowExecutionRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SignalWorkflowExecutionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SignalWorkflowExecutionRequest + switch t := that.(type) { + case *SignalWorkflowExecutionRequest: + that1 = t + case SignalWorkflowExecutionRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SignalWorkflowExecutionResponse to the protobuf v3 wire format +func (val *SignalWorkflowExecutionResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SignalWorkflowExecutionResponse from the protobuf v3 wire format +func (val *SignalWorkflowExecutionResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SignalWorkflowExecutionResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SignalWorkflowExecutionResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SignalWorkflowExecutionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SignalWorkflowExecutionResponse + switch t := that.(type) { + case *SignalWorkflowExecutionResponse: + that1 = t + case SignalWorkflowExecutionResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SignalWithStartWorkflowExecutionRequest to the protobuf v3 wire format +func (val *SignalWithStartWorkflowExecutionRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SignalWithStartWorkflowExecutionRequest from the protobuf v3 wire format +func (val *SignalWithStartWorkflowExecutionRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SignalWithStartWorkflowExecutionRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SignalWithStartWorkflowExecutionRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SignalWithStartWorkflowExecutionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SignalWithStartWorkflowExecutionRequest + switch t := that.(type) { + case *SignalWithStartWorkflowExecutionRequest: + that1 = t + case SignalWithStartWorkflowExecutionRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type SignalWithStartWorkflowExecutionResponse to the protobuf v3 wire format +func (val *SignalWithStartWorkflowExecutionResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type SignalWithStartWorkflowExecutionResponse from the protobuf v3 wire format +func (val *SignalWithStartWorkflowExecutionResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *SignalWithStartWorkflowExecutionResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two SignalWithStartWorkflowExecutionResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *SignalWithStartWorkflowExecutionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *SignalWithStartWorkflowExecutionResponse + switch t := that.(type) { + case *SignalWithStartWorkflowExecutionResponse: + that1 = t + case SignalWithStartWorkflowExecutionResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ResetWorkflowExecutionRequest to the protobuf v3 wire format +func (val *ResetWorkflowExecutionRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ResetWorkflowExecutionRequest from the protobuf v3 wire format +func (val *ResetWorkflowExecutionRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ResetWorkflowExecutionRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ResetWorkflowExecutionRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ResetWorkflowExecutionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ResetWorkflowExecutionRequest + switch t := that.(type) { + case *ResetWorkflowExecutionRequest: + that1 = t + case ResetWorkflowExecutionRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ResetWorkflowExecutionResponse to the protobuf v3 wire format +func (val *ResetWorkflowExecutionResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ResetWorkflowExecutionResponse from the protobuf v3 wire format +func (val *ResetWorkflowExecutionResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ResetWorkflowExecutionResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ResetWorkflowExecutionResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ResetWorkflowExecutionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ResetWorkflowExecutionResponse + switch t := that.(type) { + case *ResetWorkflowExecutionResponse: + that1 = t + case ResetWorkflowExecutionResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TerminateWorkflowExecutionRequest to the protobuf v3 wire format +func (val *TerminateWorkflowExecutionRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TerminateWorkflowExecutionRequest from the protobuf v3 wire format +func (val *TerminateWorkflowExecutionRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TerminateWorkflowExecutionRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TerminateWorkflowExecutionRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TerminateWorkflowExecutionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TerminateWorkflowExecutionRequest + switch t := that.(type) { + case *TerminateWorkflowExecutionRequest: + that1 = t + case TerminateWorkflowExecutionRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type TerminateWorkflowExecutionResponse to the protobuf v3 wire format +func (val *TerminateWorkflowExecutionResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type TerminateWorkflowExecutionResponse from the protobuf v3 wire format +func (val *TerminateWorkflowExecutionResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *TerminateWorkflowExecutionResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two TerminateWorkflowExecutionResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *TerminateWorkflowExecutionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *TerminateWorkflowExecutionResponse + switch t := that.(type) { + case *TerminateWorkflowExecutionResponse: + that1 = t + case TerminateWorkflowExecutionResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteWorkflowExecutionRequest to the protobuf v3 wire format +func (val *DeleteWorkflowExecutionRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteWorkflowExecutionRequest from the protobuf v3 wire format +func (val *DeleteWorkflowExecutionRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteWorkflowExecutionRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteWorkflowExecutionRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteWorkflowExecutionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteWorkflowExecutionRequest + switch t := that.(type) { + case *DeleteWorkflowExecutionRequest: + that1 = t + case DeleteWorkflowExecutionRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteWorkflowExecutionResponse to the protobuf v3 wire format +func (val *DeleteWorkflowExecutionResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteWorkflowExecutionResponse from the protobuf v3 wire format +func (val *DeleteWorkflowExecutionResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteWorkflowExecutionResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteWorkflowExecutionResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteWorkflowExecutionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteWorkflowExecutionResponse + switch t := that.(type) { + case *DeleteWorkflowExecutionResponse: + that1 = t + case DeleteWorkflowExecutionResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListOpenWorkflowExecutionsRequest to the protobuf v3 wire format +func (val *ListOpenWorkflowExecutionsRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListOpenWorkflowExecutionsRequest from the protobuf v3 wire format +func (val *ListOpenWorkflowExecutionsRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListOpenWorkflowExecutionsRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListOpenWorkflowExecutionsRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListOpenWorkflowExecutionsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListOpenWorkflowExecutionsRequest + switch t := that.(type) { + case *ListOpenWorkflowExecutionsRequest: + that1 = t + case ListOpenWorkflowExecutionsRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListOpenWorkflowExecutionsResponse to the protobuf v3 wire format +func (val *ListOpenWorkflowExecutionsResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListOpenWorkflowExecutionsResponse from the protobuf v3 wire format +func (val *ListOpenWorkflowExecutionsResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListOpenWorkflowExecutionsResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListOpenWorkflowExecutionsResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListOpenWorkflowExecutionsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListOpenWorkflowExecutionsResponse + switch t := that.(type) { + case *ListOpenWorkflowExecutionsResponse: + that1 = t + case ListOpenWorkflowExecutionsResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListClosedWorkflowExecutionsRequest to the protobuf v3 wire format +func (val *ListClosedWorkflowExecutionsRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListClosedWorkflowExecutionsRequest from the protobuf v3 wire format +func (val *ListClosedWorkflowExecutionsRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListClosedWorkflowExecutionsRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListClosedWorkflowExecutionsRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListClosedWorkflowExecutionsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListClosedWorkflowExecutionsRequest + switch t := that.(type) { + case *ListClosedWorkflowExecutionsRequest: + that1 = t + case ListClosedWorkflowExecutionsRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListClosedWorkflowExecutionsResponse to the protobuf v3 wire format +func (val *ListClosedWorkflowExecutionsResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListClosedWorkflowExecutionsResponse from the protobuf v3 wire format +func (val *ListClosedWorkflowExecutionsResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListClosedWorkflowExecutionsResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListClosedWorkflowExecutionsResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListClosedWorkflowExecutionsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListClosedWorkflowExecutionsResponse + switch t := that.(type) { + case *ListClosedWorkflowExecutionsResponse: + that1 = t + case ListClosedWorkflowExecutionsResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListWorkflowExecutionsRequest to the protobuf v3 wire format +func (val *ListWorkflowExecutionsRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListWorkflowExecutionsRequest from the protobuf v3 wire format +func (val *ListWorkflowExecutionsRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListWorkflowExecutionsRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListWorkflowExecutionsRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListWorkflowExecutionsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListWorkflowExecutionsRequest + switch t := that.(type) { + case *ListWorkflowExecutionsRequest: + that1 = t + case ListWorkflowExecutionsRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListWorkflowExecutionsResponse to the protobuf v3 wire format +func (val *ListWorkflowExecutionsResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListWorkflowExecutionsResponse from the protobuf v3 wire format +func (val *ListWorkflowExecutionsResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListWorkflowExecutionsResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListWorkflowExecutionsResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListWorkflowExecutionsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListWorkflowExecutionsResponse + switch t := that.(type) { + case *ListWorkflowExecutionsResponse: + that1 = t + case ListWorkflowExecutionsResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListArchivedWorkflowExecutionsRequest to the protobuf v3 wire format +func (val *ListArchivedWorkflowExecutionsRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListArchivedWorkflowExecutionsRequest from the protobuf v3 wire format +func (val *ListArchivedWorkflowExecutionsRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListArchivedWorkflowExecutionsRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListArchivedWorkflowExecutionsRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListArchivedWorkflowExecutionsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListArchivedWorkflowExecutionsRequest + switch t := that.(type) { + case *ListArchivedWorkflowExecutionsRequest: + that1 = t + case ListArchivedWorkflowExecutionsRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListArchivedWorkflowExecutionsResponse to the protobuf v3 wire format +func (val *ListArchivedWorkflowExecutionsResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListArchivedWorkflowExecutionsResponse from the protobuf v3 wire format +func (val *ListArchivedWorkflowExecutionsResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListArchivedWorkflowExecutionsResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListArchivedWorkflowExecutionsResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListArchivedWorkflowExecutionsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListArchivedWorkflowExecutionsResponse + switch t := that.(type) { + case *ListArchivedWorkflowExecutionsResponse: + that1 = t + case ListArchivedWorkflowExecutionsResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ScanWorkflowExecutionsRequest to the protobuf v3 wire format +func (val *ScanWorkflowExecutionsRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ScanWorkflowExecutionsRequest from the protobuf v3 wire format +func (val *ScanWorkflowExecutionsRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ScanWorkflowExecutionsRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ScanWorkflowExecutionsRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ScanWorkflowExecutionsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ScanWorkflowExecutionsRequest + switch t := that.(type) { + case *ScanWorkflowExecutionsRequest: + that1 = t + case ScanWorkflowExecutionsRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ScanWorkflowExecutionsResponse to the protobuf v3 wire format +func (val *ScanWorkflowExecutionsResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ScanWorkflowExecutionsResponse from the protobuf v3 wire format +func (val *ScanWorkflowExecutionsResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ScanWorkflowExecutionsResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ScanWorkflowExecutionsResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ScanWorkflowExecutionsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ScanWorkflowExecutionsResponse + switch t := that.(type) { + case *ScanWorkflowExecutionsResponse: + that1 = t + case ScanWorkflowExecutionsResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CountWorkflowExecutionsRequest to the protobuf v3 wire format +func (val *CountWorkflowExecutionsRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CountWorkflowExecutionsRequest from the protobuf v3 wire format +func (val *CountWorkflowExecutionsRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CountWorkflowExecutionsRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CountWorkflowExecutionsRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CountWorkflowExecutionsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CountWorkflowExecutionsRequest + switch t := that.(type) { + case *CountWorkflowExecutionsRequest: + that1 = t + case CountWorkflowExecutionsRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CountWorkflowExecutionsResponse to the protobuf v3 wire format +func (val *CountWorkflowExecutionsResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CountWorkflowExecutionsResponse from the protobuf v3 wire format +func (val *CountWorkflowExecutionsResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CountWorkflowExecutionsResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CountWorkflowExecutionsResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CountWorkflowExecutionsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CountWorkflowExecutionsResponse + switch t := that.(type) { + case *CountWorkflowExecutionsResponse: + that1 = t + case CountWorkflowExecutionsResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetSearchAttributesRequest to the protobuf v3 wire format +func (val *GetSearchAttributesRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetSearchAttributesRequest from the protobuf v3 wire format +func (val *GetSearchAttributesRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetSearchAttributesRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetSearchAttributesRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetSearchAttributesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetSearchAttributesRequest + switch t := that.(type) { + case *GetSearchAttributesRequest: + that1 = t + case GetSearchAttributesRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetSearchAttributesResponse to the protobuf v3 wire format +func (val *GetSearchAttributesResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetSearchAttributesResponse from the protobuf v3 wire format +func (val *GetSearchAttributesResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetSearchAttributesResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetSearchAttributesResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetSearchAttributesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetSearchAttributesResponse + switch t := that.(type) { + case *GetSearchAttributesResponse: + that1 = t + case GetSearchAttributesResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondQueryTaskCompletedRequest to the protobuf v3 wire format +func (val *RespondQueryTaskCompletedRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondQueryTaskCompletedRequest from the protobuf v3 wire format +func (val *RespondQueryTaskCompletedRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondQueryTaskCompletedRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondQueryTaskCompletedRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondQueryTaskCompletedRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondQueryTaskCompletedRequest + switch t := that.(type) { + case *RespondQueryTaskCompletedRequest: + that1 = t + case RespondQueryTaskCompletedRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondQueryTaskCompletedResponse to the protobuf v3 wire format +func (val *RespondQueryTaskCompletedResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondQueryTaskCompletedResponse from the protobuf v3 wire format +func (val *RespondQueryTaskCompletedResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondQueryTaskCompletedResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondQueryTaskCompletedResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondQueryTaskCompletedResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondQueryTaskCompletedResponse + switch t := that.(type) { + case *RespondQueryTaskCompletedResponse: + that1 = t + case RespondQueryTaskCompletedResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ResetStickyTaskQueueRequest to the protobuf v3 wire format +func (val *ResetStickyTaskQueueRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ResetStickyTaskQueueRequest from the protobuf v3 wire format +func (val *ResetStickyTaskQueueRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ResetStickyTaskQueueRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ResetStickyTaskQueueRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ResetStickyTaskQueueRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ResetStickyTaskQueueRequest + switch t := that.(type) { + case *ResetStickyTaskQueueRequest: + that1 = t + case ResetStickyTaskQueueRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ResetStickyTaskQueueResponse to the protobuf v3 wire format +func (val *ResetStickyTaskQueueResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ResetStickyTaskQueueResponse from the protobuf v3 wire format +func (val *ResetStickyTaskQueueResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ResetStickyTaskQueueResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ResetStickyTaskQueueResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ResetStickyTaskQueueResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ResetStickyTaskQueueResponse + switch t := that.(type) { + case *ResetStickyTaskQueueResponse: + that1 = t + case ResetStickyTaskQueueResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type QueryWorkflowRequest to the protobuf v3 wire format +func (val *QueryWorkflowRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type QueryWorkflowRequest from the protobuf v3 wire format +func (val *QueryWorkflowRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *QueryWorkflowRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two QueryWorkflowRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *QueryWorkflowRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *QueryWorkflowRequest + switch t := that.(type) { + case *QueryWorkflowRequest: + that1 = t + case QueryWorkflowRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type QueryWorkflowResponse to the protobuf v3 wire format +func (val *QueryWorkflowResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type QueryWorkflowResponse from the protobuf v3 wire format +func (val *QueryWorkflowResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *QueryWorkflowResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two QueryWorkflowResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *QueryWorkflowResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *QueryWorkflowResponse + switch t := that.(type) { + case *QueryWorkflowResponse: + that1 = t + case QueryWorkflowResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DescribeWorkflowExecutionRequest to the protobuf v3 wire format +func (val *DescribeWorkflowExecutionRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DescribeWorkflowExecutionRequest from the protobuf v3 wire format +func (val *DescribeWorkflowExecutionRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DescribeWorkflowExecutionRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DescribeWorkflowExecutionRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DescribeWorkflowExecutionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DescribeWorkflowExecutionRequest + switch t := that.(type) { + case *DescribeWorkflowExecutionRequest: + that1 = t + case DescribeWorkflowExecutionRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DescribeWorkflowExecutionResponse to the protobuf v3 wire format +func (val *DescribeWorkflowExecutionResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DescribeWorkflowExecutionResponse from the protobuf v3 wire format +func (val *DescribeWorkflowExecutionResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DescribeWorkflowExecutionResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DescribeWorkflowExecutionResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DescribeWorkflowExecutionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DescribeWorkflowExecutionResponse + switch t := that.(type) { + case *DescribeWorkflowExecutionResponse: + that1 = t + case DescribeWorkflowExecutionResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DescribeTaskQueueRequest to the protobuf v3 wire format +func (val *DescribeTaskQueueRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DescribeTaskQueueRequest from the protobuf v3 wire format +func (val *DescribeTaskQueueRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DescribeTaskQueueRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DescribeTaskQueueRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DescribeTaskQueueRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DescribeTaskQueueRequest + switch t := that.(type) { + case *DescribeTaskQueueRequest: + that1 = t + case DescribeTaskQueueRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DescribeTaskQueueResponse to the protobuf v3 wire format +func (val *DescribeTaskQueueResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DescribeTaskQueueResponse from the protobuf v3 wire format +func (val *DescribeTaskQueueResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DescribeTaskQueueResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DescribeTaskQueueResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DescribeTaskQueueResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DescribeTaskQueueResponse + switch t := that.(type) { + case *DescribeTaskQueueResponse: + that1 = t + case DescribeTaskQueueResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetClusterInfoRequest to the protobuf v3 wire format +func (val *GetClusterInfoRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetClusterInfoRequest from the protobuf v3 wire format +func (val *GetClusterInfoRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetClusterInfoRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetClusterInfoRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetClusterInfoRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetClusterInfoRequest + switch t := that.(type) { + case *GetClusterInfoRequest: + that1 = t + case GetClusterInfoRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetClusterInfoResponse to the protobuf v3 wire format +func (val *GetClusterInfoResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetClusterInfoResponse from the protobuf v3 wire format +func (val *GetClusterInfoResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetClusterInfoResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetClusterInfoResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetClusterInfoResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetClusterInfoResponse + switch t := that.(type) { + case *GetClusterInfoResponse: + that1 = t + case GetClusterInfoResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetSystemInfoRequest to the protobuf v3 wire format +func (val *GetSystemInfoRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetSystemInfoRequest from the protobuf v3 wire format +func (val *GetSystemInfoRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetSystemInfoRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetSystemInfoRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetSystemInfoRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetSystemInfoRequest + switch t := that.(type) { + case *GetSystemInfoRequest: + that1 = t + case GetSystemInfoRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetSystemInfoResponse to the protobuf v3 wire format +func (val *GetSystemInfoResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetSystemInfoResponse from the protobuf v3 wire format +func (val *GetSystemInfoResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetSystemInfoResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetSystemInfoResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetSystemInfoResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetSystemInfoResponse + switch t := that.(type) { + case *GetSystemInfoResponse: + that1 = t + case GetSystemInfoResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListTaskQueuePartitionsRequest to the protobuf v3 wire format +func (val *ListTaskQueuePartitionsRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListTaskQueuePartitionsRequest from the protobuf v3 wire format +func (val *ListTaskQueuePartitionsRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListTaskQueuePartitionsRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListTaskQueuePartitionsRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListTaskQueuePartitionsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListTaskQueuePartitionsRequest + switch t := that.(type) { + case *ListTaskQueuePartitionsRequest: + that1 = t + case ListTaskQueuePartitionsRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListTaskQueuePartitionsResponse to the protobuf v3 wire format +func (val *ListTaskQueuePartitionsResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListTaskQueuePartitionsResponse from the protobuf v3 wire format +func (val *ListTaskQueuePartitionsResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListTaskQueuePartitionsResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListTaskQueuePartitionsResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListTaskQueuePartitionsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListTaskQueuePartitionsResponse + switch t := that.(type) { + case *ListTaskQueuePartitionsResponse: + that1 = t + case ListTaskQueuePartitionsResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateScheduleRequest to the protobuf v3 wire format +func (val *CreateScheduleRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateScheduleRequest from the protobuf v3 wire format +func (val *CreateScheduleRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateScheduleRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateScheduleRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateScheduleRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateScheduleRequest + switch t := that.(type) { + case *CreateScheduleRequest: + that1 = t + case CreateScheduleRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type CreateScheduleResponse to the protobuf v3 wire format +func (val *CreateScheduleResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type CreateScheduleResponse from the protobuf v3 wire format +func (val *CreateScheduleResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *CreateScheduleResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two CreateScheduleResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *CreateScheduleResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *CreateScheduleResponse + switch t := that.(type) { + case *CreateScheduleResponse: + that1 = t + case CreateScheduleResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DescribeScheduleRequest to the protobuf v3 wire format +func (val *DescribeScheduleRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DescribeScheduleRequest from the protobuf v3 wire format +func (val *DescribeScheduleRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DescribeScheduleRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DescribeScheduleRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DescribeScheduleRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DescribeScheduleRequest + switch t := that.(type) { + case *DescribeScheduleRequest: + that1 = t + case DescribeScheduleRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DescribeScheduleResponse to the protobuf v3 wire format +func (val *DescribeScheduleResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DescribeScheduleResponse from the protobuf v3 wire format +func (val *DescribeScheduleResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DescribeScheduleResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DescribeScheduleResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DescribeScheduleResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DescribeScheduleResponse + switch t := that.(type) { + case *DescribeScheduleResponse: + that1 = t + case DescribeScheduleResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateScheduleRequest to the protobuf v3 wire format +func (val *UpdateScheduleRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateScheduleRequest from the protobuf v3 wire format +func (val *UpdateScheduleRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateScheduleRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateScheduleRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateScheduleRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateScheduleRequest + switch t := that.(type) { + case *UpdateScheduleRequest: + that1 = t + case UpdateScheduleRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateScheduleResponse to the protobuf v3 wire format +func (val *UpdateScheduleResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateScheduleResponse from the protobuf v3 wire format +func (val *UpdateScheduleResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateScheduleResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateScheduleResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateScheduleResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateScheduleResponse + switch t := that.(type) { + case *UpdateScheduleResponse: + that1 = t + case UpdateScheduleResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PatchScheduleRequest to the protobuf v3 wire format +func (val *PatchScheduleRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PatchScheduleRequest from the protobuf v3 wire format +func (val *PatchScheduleRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PatchScheduleRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PatchScheduleRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PatchScheduleRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PatchScheduleRequest + switch t := that.(type) { + case *PatchScheduleRequest: + that1 = t + case PatchScheduleRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PatchScheduleResponse to the protobuf v3 wire format +func (val *PatchScheduleResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PatchScheduleResponse from the protobuf v3 wire format +func (val *PatchScheduleResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PatchScheduleResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PatchScheduleResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PatchScheduleResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PatchScheduleResponse + switch t := that.(type) { + case *PatchScheduleResponse: + that1 = t + case PatchScheduleResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListScheduleMatchingTimesRequest to the protobuf v3 wire format +func (val *ListScheduleMatchingTimesRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListScheduleMatchingTimesRequest from the protobuf v3 wire format +func (val *ListScheduleMatchingTimesRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListScheduleMatchingTimesRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListScheduleMatchingTimesRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListScheduleMatchingTimesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListScheduleMatchingTimesRequest + switch t := that.(type) { + case *ListScheduleMatchingTimesRequest: + that1 = t + case ListScheduleMatchingTimesRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListScheduleMatchingTimesResponse to the protobuf v3 wire format +func (val *ListScheduleMatchingTimesResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListScheduleMatchingTimesResponse from the protobuf v3 wire format +func (val *ListScheduleMatchingTimesResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListScheduleMatchingTimesResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListScheduleMatchingTimesResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListScheduleMatchingTimesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListScheduleMatchingTimesResponse + switch t := that.(type) { + case *ListScheduleMatchingTimesResponse: + that1 = t + case ListScheduleMatchingTimesResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteScheduleRequest to the protobuf v3 wire format +func (val *DeleteScheduleRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteScheduleRequest from the protobuf v3 wire format +func (val *DeleteScheduleRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteScheduleRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteScheduleRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteScheduleRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteScheduleRequest + switch t := that.(type) { + case *DeleteScheduleRequest: + that1 = t + case DeleteScheduleRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DeleteScheduleResponse to the protobuf v3 wire format +func (val *DeleteScheduleResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DeleteScheduleResponse from the protobuf v3 wire format +func (val *DeleteScheduleResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DeleteScheduleResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DeleteScheduleResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DeleteScheduleResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DeleteScheduleResponse + switch t := that.(type) { + case *DeleteScheduleResponse: + that1 = t + case DeleteScheduleResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListSchedulesRequest to the protobuf v3 wire format +func (val *ListSchedulesRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListSchedulesRequest from the protobuf v3 wire format +func (val *ListSchedulesRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListSchedulesRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListSchedulesRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListSchedulesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListSchedulesRequest + switch t := that.(type) { + case *ListSchedulesRequest: + that1 = t + case ListSchedulesRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListSchedulesResponse to the protobuf v3 wire format +func (val *ListSchedulesResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListSchedulesResponse from the protobuf v3 wire format +func (val *ListSchedulesResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListSchedulesResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListSchedulesResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListSchedulesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListSchedulesResponse + switch t := that.(type) { + case *ListSchedulesResponse: + that1 = t + case ListSchedulesResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateWorkerBuildIdCompatibilityRequest to the protobuf v3 wire format +func (val *UpdateWorkerBuildIdCompatibilityRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateWorkerBuildIdCompatibilityRequest from the protobuf v3 wire format +func (val *UpdateWorkerBuildIdCompatibilityRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateWorkerBuildIdCompatibilityRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateWorkerBuildIdCompatibilityRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateWorkerBuildIdCompatibilityRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateWorkerBuildIdCompatibilityRequest + switch t := that.(type) { + case *UpdateWorkerBuildIdCompatibilityRequest: + that1 = t + case UpdateWorkerBuildIdCompatibilityRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateWorkerBuildIdCompatibilityResponse to the protobuf v3 wire format +func (val *UpdateWorkerBuildIdCompatibilityResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateWorkerBuildIdCompatibilityResponse from the protobuf v3 wire format +func (val *UpdateWorkerBuildIdCompatibilityResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateWorkerBuildIdCompatibilityResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateWorkerBuildIdCompatibilityResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateWorkerBuildIdCompatibilityResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateWorkerBuildIdCompatibilityResponse + switch t := that.(type) { + case *UpdateWorkerBuildIdCompatibilityResponse: + that1 = t + case UpdateWorkerBuildIdCompatibilityResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetWorkerBuildIdCompatibilityRequest to the protobuf v3 wire format +func (val *GetWorkerBuildIdCompatibilityRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetWorkerBuildIdCompatibilityRequest from the protobuf v3 wire format +func (val *GetWorkerBuildIdCompatibilityRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetWorkerBuildIdCompatibilityRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetWorkerBuildIdCompatibilityRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetWorkerBuildIdCompatibilityRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetWorkerBuildIdCompatibilityRequest + switch t := that.(type) { + case *GetWorkerBuildIdCompatibilityRequest: + that1 = t + case GetWorkerBuildIdCompatibilityRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetWorkerBuildIdCompatibilityResponse to the protobuf v3 wire format +func (val *GetWorkerBuildIdCompatibilityResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetWorkerBuildIdCompatibilityResponse from the protobuf v3 wire format +func (val *GetWorkerBuildIdCompatibilityResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetWorkerBuildIdCompatibilityResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetWorkerBuildIdCompatibilityResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetWorkerBuildIdCompatibilityResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetWorkerBuildIdCompatibilityResponse + switch t := that.(type) { + case *GetWorkerBuildIdCompatibilityResponse: + that1 = t + case GetWorkerBuildIdCompatibilityResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateWorkerVersioningRulesRequest to the protobuf v3 wire format +func (val *UpdateWorkerVersioningRulesRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateWorkerVersioningRulesRequest from the protobuf v3 wire format +func (val *UpdateWorkerVersioningRulesRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateWorkerVersioningRulesRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateWorkerVersioningRulesRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateWorkerVersioningRulesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateWorkerVersioningRulesRequest + switch t := that.(type) { + case *UpdateWorkerVersioningRulesRequest: + that1 = t + case UpdateWorkerVersioningRulesRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateWorkerVersioningRulesResponse to the protobuf v3 wire format +func (val *UpdateWorkerVersioningRulesResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateWorkerVersioningRulesResponse from the protobuf v3 wire format +func (val *UpdateWorkerVersioningRulesResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateWorkerVersioningRulesResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateWorkerVersioningRulesResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateWorkerVersioningRulesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateWorkerVersioningRulesResponse + switch t := that.(type) { + case *UpdateWorkerVersioningRulesResponse: + that1 = t + case UpdateWorkerVersioningRulesResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetWorkerVersioningRulesRequest to the protobuf v3 wire format +func (val *GetWorkerVersioningRulesRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetWorkerVersioningRulesRequest from the protobuf v3 wire format +func (val *GetWorkerVersioningRulesRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetWorkerVersioningRulesRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetWorkerVersioningRulesRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetWorkerVersioningRulesRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetWorkerVersioningRulesRequest + switch t := that.(type) { + case *GetWorkerVersioningRulesRequest: + that1 = t + case GetWorkerVersioningRulesRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetWorkerVersioningRulesResponse to the protobuf v3 wire format +func (val *GetWorkerVersioningRulesResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetWorkerVersioningRulesResponse from the protobuf v3 wire format +func (val *GetWorkerVersioningRulesResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetWorkerVersioningRulesResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetWorkerVersioningRulesResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetWorkerVersioningRulesResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetWorkerVersioningRulesResponse + switch t := that.(type) { + case *GetWorkerVersioningRulesResponse: + that1 = t + case GetWorkerVersioningRulesResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetWorkerTaskReachabilityRequest to the protobuf v3 wire format +func (val *GetWorkerTaskReachabilityRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetWorkerTaskReachabilityRequest from the protobuf v3 wire format +func (val *GetWorkerTaskReachabilityRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetWorkerTaskReachabilityRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetWorkerTaskReachabilityRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetWorkerTaskReachabilityRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetWorkerTaskReachabilityRequest + switch t := that.(type) { + case *GetWorkerTaskReachabilityRequest: + that1 = t + case GetWorkerTaskReachabilityRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type GetWorkerTaskReachabilityResponse to the protobuf v3 wire format +func (val *GetWorkerTaskReachabilityResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type GetWorkerTaskReachabilityResponse from the protobuf v3 wire format +func (val *GetWorkerTaskReachabilityResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *GetWorkerTaskReachabilityResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two GetWorkerTaskReachabilityResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *GetWorkerTaskReachabilityResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *GetWorkerTaskReachabilityResponse + switch t := that.(type) { + case *GetWorkerTaskReachabilityResponse: + that1 = t + case GetWorkerTaskReachabilityResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateWorkflowExecutionRequest to the protobuf v3 wire format +func (val *UpdateWorkflowExecutionRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateWorkflowExecutionRequest from the protobuf v3 wire format +func (val *UpdateWorkflowExecutionRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateWorkflowExecutionRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateWorkflowExecutionRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateWorkflowExecutionRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateWorkflowExecutionRequest + switch t := that.(type) { + case *UpdateWorkflowExecutionRequest: + that1 = t + case UpdateWorkflowExecutionRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type UpdateWorkflowExecutionResponse to the protobuf v3 wire format +func (val *UpdateWorkflowExecutionResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type UpdateWorkflowExecutionResponse from the protobuf v3 wire format +func (val *UpdateWorkflowExecutionResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *UpdateWorkflowExecutionResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two UpdateWorkflowExecutionResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *UpdateWorkflowExecutionResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *UpdateWorkflowExecutionResponse + switch t := that.(type) { + case *UpdateWorkflowExecutionResponse: + that1 = t + case UpdateWorkflowExecutionResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StartBatchOperationRequest to the protobuf v3 wire format +func (val *StartBatchOperationRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StartBatchOperationRequest from the protobuf v3 wire format +func (val *StartBatchOperationRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StartBatchOperationRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StartBatchOperationRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StartBatchOperationRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StartBatchOperationRequest + switch t := that.(type) { + case *StartBatchOperationRequest: + that1 = t + case StartBatchOperationRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StartBatchOperationResponse to the protobuf v3 wire format +func (val *StartBatchOperationResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StartBatchOperationResponse from the protobuf v3 wire format +func (val *StartBatchOperationResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StartBatchOperationResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StartBatchOperationResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StartBatchOperationResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StartBatchOperationResponse + switch t := that.(type) { + case *StartBatchOperationResponse: + that1 = t + case StartBatchOperationResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StopBatchOperationRequest to the protobuf v3 wire format +func (val *StopBatchOperationRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StopBatchOperationRequest from the protobuf v3 wire format +func (val *StopBatchOperationRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StopBatchOperationRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StopBatchOperationRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StopBatchOperationRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StopBatchOperationRequest + switch t := that.(type) { + case *StopBatchOperationRequest: + that1 = t + case StopBatchOperationRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type StopBatchOperationResponse to the protobuf v3 wire format +func (val *StopBatchOperationResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type StopBatchOperationResponse from the protobuf v3 wire format +func (val *StopBatchOperationResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *StopBatchOperationResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two StopBatchOperationResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *StopBatchOperationResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *StopBatchOperationResponse + switch t := that.(type) { + case *StopBatchOperationResponse: + that1 = t + case StopBatchOperationResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DescribeBatchOperationRequest to the protobuf v3 wire format +func (val *DescribeBatchOperationRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DescribeBatchOperationRequest from the protobuf v3 wire format +func (val *DescribeBatchOperationRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DescribeBatchOperationRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DescribeBatchOperationRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DescribeBatchOperationRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DescribeBatchOperationRequest + switch t := that.(type) { + case *DescribeBatchOperationRequest: + that1 = t + case DescribeBatchOperationRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type DescribeBatchOperationResponse to the protobuf v3 wire format +func (val *DescribeBatchOperationResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type DescribeBatchOperationResponse from the protobuf v3 wire format +func (val *DescribeBatchOperationResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *DescribeBatchOperationResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two DescribeBatchOperationResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *DescribeBatchOperationResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *DescribeBatchOperationResponse + switch t := that.(type) { + case *DescribeBatchOperationResponse: + that1 = t + case DescribeBatchOperationResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListBatchOperationsRequest to the protobuf v3 wire format +func (val *ListBatchOperationsRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListBatchOperationsRequest from the protobuf v3 wire format +func (val *ListBatchOperationsRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListBatchOperationsRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListBatchOperationsRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListBatchOperationsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListBatchOperationsRequest + switch t := that.(type) { + case *ListBatchOperationsRequest: + that1 = t + case ListBatchOperationsRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ListBatchOperationsResponse to the protobuf v3 wire format +func (val *ListBatchOperationsResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ListBatchOperationsResponse from the protobuf v3 wire format +func (val *ListBatchOperationsResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ListBatchOperationsResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ListBatchOperationsResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ListBatchOperationsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ListBatchOperationsResponse + switch t := that.(type) { + case *ListBatchOperationsResponse: + that1 = t + case ListBatchOperationsResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PollWorkflowExecutionUpdateRequest to the protobuf v3 wire format +func (val *PollWorkflowExecutionUpdateRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PollWorkflowExecutionUpdateRequest from the protobuf v3 wire format +func (val *PollWorkflowExecutionUpdateRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PollWorkflowExecutionUpdateRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PollWorkflowExecutionUpdateRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PollWorkflowExecutionUpdateRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PollWorkflowExecutionUpdateRequest + switch t := that.(type) { + case *PollWorkflowExecutionUpdateRequest: + that1 = t + case PollWorkflowExecutionUpdateRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PollWorkflowExecutionUpdateResponse to the protobuf v3 wire format +func (val *PollWorkflowExecutionUpdateResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PollWorkflowExecutionUpdateResponse from the protobuf v3 wire format +func (val *PollWorkflowExecutionUpdateResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PollWorkflowExecutionUpdateResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PollWorkflowExecutionUpdateResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PollWorkflowExecutionUpdateResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PollWorkflowExecutionUpdateResponse + switch t := that.(type) { + case *PollWorkflowExecutionUpdateResponse: + that1 = t + case PollWorkflowExecutionUpdateResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PollNexusTaskQueueRequest to the protobuf v3 wire format +func (val *PollNexusTaskQueueRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PollNexusTaskQueueRequest from the protobuf v3 wire format +func (val *PollNexusTaskQueueRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PollNexusTaskQueueRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PollNexusTaskQueueRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PollNexusTaskQueueRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PollNexusTaskQueueRequest + switch t := that.(type) { + case *PollNexusTaskQueueRequest: + that1 = t + case PollNexusTaskQueueRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type PollNexusTaskQueueResponse to the protobuf v3 wire format +func (val *PollNexusTaskQueueResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type PollNexusTaskQueueResponse from the protobuf v3 wire format +func (val *PollNexusTaskQueueResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *PollNexusTaskQueueResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two PollNexusTaskQueueResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *PollNexusTaskQueueResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *PollNexusTaskQueueResponse + switch t := that.(type) { + case *PollNexusTaskQueueResponse: + that1 = t + case PollNexusTaskQueueResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondNexusTaskCompletedRequest to the protobuf v3 wire format +func (val *RespondNexusTaskCompletedRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondNexusTaskCompletedRequest from the protobuf v3 wire format +func (val *RespondNexusTaskCompletedRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondNexusTaskCompletedRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondNexusTaskCompletedRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondNexusTaskCompletedRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondNexusTaskCompletedRequest + switch t := that.(type) { + case *RespondNexusTaskCompletedRequest: + that1 = t + case RespondNexusTaskCompletedRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondNexusTaskCompletedResponse to the protobuf v3 wire format +func (val *RespondNexusTaskCompletedResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondNexusTaskCompletedResponse from the protobuf v3 wire format +func (val *RespondNexusTaskCompletedResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondNexusTaskCompletedResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondNexusTaskCompletedResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondNexusTaskCompletedResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondNexusTaskCompletedResponse + switch t := that.(type) { + case *RespondNexusTaskCompletedResponse: + that1 = t + case RespondNexusTaskCompletedResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondNexusTaskFailedRequest to the protobuf v3 wire format +func (val *RespondNexusTaskFailedRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondNexusTaskFailedRequest from the protobuf v3 wire format +func (val *RespondNexusTaskFailedRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondNexusTaskFailedRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondNexusTaskFailedRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondNexusTaskFailedRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondNexusTaskFailedRequest + switch t := that.(type) { + case *RespondNexusTaskFailedRequest: + that1 = t + case RespondNexusTaskFailedRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type RespondNexusTaskFailedResponse to the protobuf v3 wire format +func (val *RespondNexusTaskFailedResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type RespondNexusTaskFailedResponse from the protobuf v3 wire format +func (val *RespondNexusTaskFailedResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *RespondNexusTaskFailedResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two RespondNexusTaskFailedResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *RespondNexusTaskFailedResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *RespondNexusTaskFailedResponse + switch t := that.(type) { + case *RespondNexusTaskFailedResponse: + that1 = t + case RespondNexusTaskFailedResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ExecuteMultiOperationRequest to the protobuf v3 wire format +func (val *ExecuteMultiOperationRequest) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ExecuteMultiOperationRequest from the protobuf v3 wire format +func (val *ExecuteMultiOperationRequest) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ExecuteMultiOperationRequest) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ExecuteMultiOperationRequest values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ExecuteMultiOperationRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ExecuteMultiOperationRequest + switch t := that.(type) { + case *ExecuteMultiOperationRequest: + that1 = t + case ExecuteMultiOperationRequest: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} + +// Marshal an object of type ExecuteMultiOperationResponse to the protobuf v3 wire format +func (val *ExecuteMultiOperationResponse) Marshal() ([]byte, error) { + return proto.Marshal(val) +} + +// Unmarshal an object of type ExecuteMultiOperationResponse from the protobuf v3 wire format +func (val *ExecuteMultiOperationResponse) Unmarshal(buf []byte) error { + return proto.Unmarshal(buf, val) +} + +// Size returns the size of the object, in bytes, once serialized +func (val *ExecuteMultiOperationResponse) Size() int { + return proto.Size(val) +} + +// Equal returns whether two ExecuteMultiOperationResponse values are equivalent by recursively +// comparing the message's fields. +// For more information see the documentation for +// https://pkg.go.dev/google.golang.org/protobuf/proto#Equal +func (this *ExecuteMultiOperationResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + var that1 *ExecuteMultiOperationResponse + switch t := that.(type) { + case *ExecuteMultiOperationResponse: + that1 = t + case ExecuteMultiOperationResponse: + that1 = &t + default: + return false + } + + return proto.Equal(this, that1) +} diff --git a/vendor/go.temporal.io/api/workflowservice/v1/request_response.pb.go b/vendor/go.temporal.io/api/workflowservice/v1/request_response.pb.go new file mode 100644 index 00000000000..d2ab0111c85 --- /dev/null +++ b/vendor/go.temporal.io/api/workflowservice/v1/request_response.pb.go @@ -0,0 +1,15019 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/workflowservice/v1/request_response.proto + +package workflowservice + +import ( + reflect "reflect" + sync "sync" + + v116 "go.temporal.io/api/batch/v1" + v110 "go.temporal.io/api/command/v1" + v13 "go.temporal.io/api/common/v1" + v11 "go.temporal.io/api/enums/v1" + v15 "go.temporal.io/api/failure/v1" + v111 "go.temporal.io/api/filter/v1" + v17 "go.temporal.io/api/history/v1" + v12 "go.temporal.io/api/namespace/v1" + v117 "go.temporal.io/api/nexus/v1" + v19 "go.temporal.io/api/protocol/v1" + v18 "go.temporal.io/api/query/v1" + v1 "go.temporal.io/api/replication/v1" + v114 "go.temporal.io/api/schedule/v1" + v16 "go.temporal.io/api/sdk/v1" + v14 "go.temporal.io/api/taskqueue/v1" + v115 "go.temporal.io/api/update/v1" + v113 "go.temporal.io/api/version/v1" + v112 "go.temporal.io/api/workflow/v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + durationpb "google.golang.org/protobuf/types/known/durationpb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type RegisterNamespaceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + Description string `protobuf:"bytes,2,opt,name=description,proto3" json:"description,omitempty"` + OwnerEmail string `protobuf:"bytes,3,opt,name=owner_email,json=ownerEmail,proto3" json:"owner_email,omitempty"` + WorkflowExecutionRetentionPeriod *durationpb.Duration `protobuf:"bytes,4,opt,name=workflow_execution_retention_period,json=workflowExecutionRetentionPeriod,proto3" json:"workflow_execution_retention_period,omitempty"` + Clusters []*v1.ClusterReplicationConfig `protobuf:"bytes,5,rep,name=clusters,proto3" json:"clusters,omitempty"` + ActiveClusterName string `protobuf:"bytes,6,opt,name=active_cluster_name,json=activeClusterName,proto3" json:"active_cluster_name,omitempty"` + // A key-value map for any customized purpose. + Data map[string]string `protobuf:"bytes,7,rep,name=data,proto3" json:"data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + SecurityToken string `protobuf:"bytes,8,opt,name=security_token,json=securityToken,proto3" json:"security_token,omitempty"` + IsGlobalNamespace bool `protobuf:"varint,9,opt,name=is_global_namespace,json=isGlobalNamespace,proto3" json:"is_global_namespace,omitempty"` + // If unspecified (ARCHIVAL_STATE_UNSPECIFIED) then default server configuration is used. + HistoryArchivalState v11.ArchivalState `protobuf:"varint,10,opt,name=history_archival_state,json=historyArchivalState,proto3,enum=temporal.api.enums.v1.ArchivalState" json:"history_archival_state,omitempty"` + HistoryArchivalUri string `protobuf:"bytes,11,opt,name=history_archival_uri,json=historyArchivalUri,proto3" json:"history_archival_uri,omitempty"` + // If unspecified (ARCHIVAL_STATE_UNSPECIFIED) then default server configuration is used. + VisibilityArchivalState v11.ArchivalState `protobuf:"varint,12,opt,name=visibility_archival_state,json=visibilityArchivalState,proto3,enum=temporal.api.enums.v1.ArchivalState" json:"visibility_archival_state,omitempty"` + VisibilityArchivalUri string `protobuf:"bytes,13,opt,name=visibility_archival_uri,json=visibilityArchivalUri,proto3" json:"visibility_archival_uri,omitempty"` +} + +func (x *RegisterNamespaceRequest) Reset() { + *x = RegisterNamespaceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RegisterNamespaceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RegisterNamespaceRequest) ProtoMessage() {} + +func (x *RegisterNamespaceRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RegisterNamespaceRequest.ProtoReflect.Descriptor instead. +func (*RegisterNamespaceRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{0} +} + +func (x *RegisterNamespaceRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RegisterNamespaceRequest) GetDescription() string { + if x != nil { + return x.Description + } + return "" +} + +func (x *RegisterNamespaceRequest) GetOwnerEmail() string { + if x != nil { + return x.OwnerEmail + } + return "" +} + +func (x *RegisterNamespaceRequest) GetWorkflowExecutionRetentionPeriod() *durationpb.Duration { + if x != nil { + return x.WorkflowExecutionRetentionPeriod + } + return nil +} + +func (x *RegisterNamespaceRequest) GetClusters() []*v1.ClusterReplicationConfig { + if x != nil { + return x.Clusters + } + return nil +} + +func (x *RegisterNamespaceRequest) GetActiveClusterName() string { + if x != nil { + return x.ActiveClusterName + } + return "" +} + +func (x *RegisterNamespaceRequest) GetData() map[string]string { + if x != nil { + return x.Data + } + return nil +} + +func (x *RegisterNamespaceRequest) GetSecurityToken() string { + if x != nil { + return x.SecurityToken + } + return "" +} + +func (x *RegisterNamespaceRequest) GetIsGlobalNamespace() bool { + if x != nil { + return x.IsGlobalNamespace + } + return false +} + +func (x *RegisterNamespaceRequest) GetHistoryArchivalState() v11.ArchivalState { + if x != nil { + return x.HistoryArchivalState + } + return v11.ArchivalState(0) +} + +func (x *RegisterNamespaceRequest) GetHistoryArchivalUri() string { + if x != nil { + return x.HistoryArchivalUri + } + return "" +} + +func (x *RegisterNamespaceRequest) GetVisibilityArchivalState() v11.ArchivalState { + if x != nil { + return x.VisibilityArchivalState + } + return v11.ArchivalState(0) +} + +func (x *RegisterNamespaceRequest) GetVisibilityArchivalUri() string { + if x != nil { + return x.VisibilityArchivalUri + } + return "" +} + +type RegisterNamespaceResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RegisterNamespaceResponse) Reset() { + *x = RegisterNamespaceResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RegisterNamespaceResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RegisterNamespaceResponse) ProtoMessage() {} + +func (x *RegisterNamespaceResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RegisterNamespaceResponse.ProtoReflect.Descriptor instead. +func (*RegisterNamespaceResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{1} +} + +type ListNamespacesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PageSize int32 `protobuf:"varint,1,opt,name=page_size,json=pageSize,proto3" json:"page_size,omitempty"` + NextPageToken []byte `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` + NamespaceFilter *v12.NamespaceFilter `protobuf:"bytes,3,opt,name=namespace_filter,json=namespaceFilter,proto3" json:"namespace_filter,omitempty"` +} + +func (x *ListNamespacesRequest) Reset() { + *x = ListNamespacesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListNamespacesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListNamespacesRequest) ProtoMessage() {} + +func (x *ListNamespacesRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListNamespacesRequest.ProtoReflect.Descriptor instead. +func (*ListNamespacesRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{2} +} + +func (x *ListNamespacesRequest) GetPageSize() int32 { + if x != nil { + return x.PageSize + } + return 0 +} + +func (x *ListNamespacesRequest) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +func (x *ListNamespacesRequest) GetNamespaceFilter() *v12.NamespaceFilter { + if x != nil { + return x.NamespaceFilter + } + return nil +} + +type ListNamespacesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespaces []*DescribeNamespaceResponse `protobuf:"bytes,1,rep,name=namespaces,proto3" json:"namespaces,omitempty"` + NextPageToken []byte `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *ListNamespacesResponse) Reset() { + *x = ListNamespacesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListNamespacesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListNamespacesResponse) ProtoMessage() {} + +func (x *ListNamespacesResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListNamespacesResponse.ProtoReflect.Descriptor instead. +func (*ListNamespacesResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{3} +} + +func (x *ListNamespacesResponse) GetNamespaces() []*DescribeNamespaceResponse { + if x != nil { + return x.Namespaces + } + return nil +} + +func (x *ListNamespacesResponse) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +type DescribeNamespaceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + Id string `protobuf:"bytes,2,opt,name=id,proto3" json:"id,omitempty"` +} + +func (x *DescribeNamespaceRequest) Reset() { + *x = DescribeNamespaceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeNamespaceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeNamespaceRequest) ProtoMessage() {} + +func (x *DescribeNamespaceRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DescribeNamespaceRequest.ProtoReflect.Descriptor instead. +func (*DescribeNamespaceRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{4} +} + +func (x *DescribeNamespaceRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *DescribeNamespaceRequest) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +type DescribeNamespaceResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + NamespaceInfo *v12.NamespaceInfo `protobuf:"bytes,1,opt,name=namespace_info,json=namespaceInfo,proto3" json:"namespace_info,omitempty"` + Config *v12.NamespaceConfig `protobuf:"bytes,2,opt,name=config,proto3" json:"config,omitempty"` + ReplicationConfig *v1.NamespaceReplicationConfig `protobuf:"bytes,3,opt,name=replication_config,json=replicationConfig,proto3" json:"replication_config,omitempty"` + FailoverVersion int64 `protobuf:"varint,4,opt,name=failover_version,json=failoverVersion,proto3" json:"failover_version,omitempty"` + IsGlobalNamespace bool `protobuf:"varint,5,opt,name=is_global_namespace,json=isGlobalNamespace,proto3" json:"is_global_namespace,omitempty"` + // Contains the historical state of failover_versions for the cluster, truncated to contain only the last N + // states to ensure that the list does not grow unbounded. + FailoverHistory []*v1.FailoverStatus `protobuf:"bytes,6,rep,name=failover_history,json=failoverHistory,proto3" json:"failover_history,omitempty"` +} + +func (x *DescribeNamespaceResponse) Reset() { + *x = DescribeNamespaceResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeNamespaceResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeNamespaceResponse) ProtoMessage() {} + +func (x *DescribeNamespaceResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DescribeNamespaceResponse.ProtoReflect.Descriptor instead. +func (*DescribeNamespaceResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{5} +} + +func (x *DescribeNamespaceResponse) GetNamespaceInfo() *v12.NamespaceInfo { + if x != nil { + return x.NamespaceInfo + } + return nil +} + +func (x *DescribeNamespaceResponse) GetConfig() *v12.NamespaceConfig { + if x != nil { + return x.Config + } + return nil +} + +func (x *DescribeNamespaceResponse) GetReplicationConfig() *v1.NamespaceReplicationConfig { + if x != nil { + return x.ReplicationConfig + } + return nil +} + +func (x *DescribeNamespaceResponse) GetFailoverVersion() int64 { + if x != nil { + return x.FailoverVersion + } + return 0 +} + +func (x *DescribeNamespaceResponse) GetIsGlobalNamespace() bool { + if x != nil { + return x.IsGlobalNamespace + } + return false +} + +func (x *DescribeNamespaceResponse) GetFailoverHistory() []*v1.FailoverStatus { + if x != nil { + return x.FailoverHistory + } + return nil +} + +type UpdateNamespaceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + UpdateInfo *v12.UpdateNamespaceInfo `protobuf:"bytes,2,opt,name=update_info,json=updateInfo,proto3" json:"update_info,omitempty"` + Config *v12.NamespaceConfig `protobuf:"bytes,3,opt,name=config,proto3" json:"config,omitempty"` + ReplicationConfig *v1.NamespaceReplicationConfig `protobuf:"bytes,4,opt,name=replication_config,json=replicationConfig,proto3" json:"replication_config,omitempty"` + SecurityToken string `protobuf:"bytes,5,opt,name=security_token,json=securityToken,proto3" json:"security_token,omitempty"` + DeleteBadBinary string `protobuf:"bytes,6,opt,name=delete_bad_binary,json=deleteBadBinary,proto3" json:"delete_bad_binary,omitempty"` + // promote local namespace to global namespace. Ignored if namespace is already global namespace. + PromoteNamespace bool `protobuf:"varint,7,opt,name=promote_namespace,json=promoteNamespace,proto3" json:"promote_namespace,omitempty"` +} + +func (x *UpdateNamespaceRequest) Reset() { + *x = UpdateNamespaceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateNamespaceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateNamespaceRequest) ProtoMessage() {} + +func (x *UpdateNamespaceRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateNamespaceRequest.ProtoReflect.Descriptor instead. +func (*UpdateNamespaceRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{6} +} + +func (x *UpdateNamespaceRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *UpdateNamespaceRequest) GetUpdateInfo() *v12.UpdateNamespaceInfo { + if x != nil { + return x.UpdateInfo + } + return nil +} + +func (x *UpdateNamespaceRequest) GetConfig() *v12.NamespaceConfig { + if x != nil { + return x.Config + } + return nil +} + +func (x *UpdateNamespaceRequest) GetReplicationConfig() *v1.NamespaceReplicationConfig { + if x != nil { + return x.ReplicationConfig + } + return nil +} + +func (x *UpdateNamespaceRequest) GetSecurityToken() string { + if x != nil { + return x.SecurityToken + } + return "" +} + +func (x *UpdateNamespaceRequest) GetDeleteBadBinary() string { + if x != nil { + return x.DeleteBadBinary + } + return "" +} + +func (x *UpdateNamespaceRequest) GetPromoteNamespace() bool { + if x != nil { + return x.PromoteNamespace + } + return false +} + +type UpdateNamespaceResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + NamespaceInfo *v12.NamespaceInfo `protobuf:"bytes,1,opt,name=namespace_info,json=namespaceInfo,proto3" json:"namespace_info,omitempty"` + Config *v12.NamespaceConfig `protobuf:"bytes,2,opt,name=config,proto3" json:"config,omitempty"` + ReplicationConfig *v1.NamespaceReplicationConfig `protobuf:"bytes,3,opt,name=replication_config,json=replicationConfig,proto3" json:"replication_config,omitempty"` + FailoverVersion int64 `protobuf:"varint,4,opt,name=failover_version,json=failoverVersion,proto3" json:"failover_version,omitempty"` + IsGlobalNamespace bool `protobuf:"varint,5,opt,name=is_global_namespace,json=isGlobalNamespace,proto3" json:"is_global_namespace,omitempty"` +} + +func (x *UpdateNamespaceResponse) Reset() { + *x = UpdateNamespaceResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateNamespaceResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateNamespaceResponse) ProtoMessage() {} + +func (x *UpdateNamespaceResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateNamespaceResponse.ProtoReflect.Descriptor instead. +func (*UpdateNamespaceResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{7} +} + +func (x *UpdateNamespaceResponse) GetNamespaceInfo() *v12.NamespaceInfo { + if x != nil { + return x.NamespaceInfo + } + return nil +} + +func (x *UpdateNamespaceResponse) GetConfig() *v12.NamespaceConfig { + if x != nil { + return x.Config + } + return nil +} + +func (x *UpdateNamespaceResponse) GetReplicationConfig() *v1.NamespaceReplicationConfig { + if x != nil { + return x.ReplicationConfig + } + return nil +} + +func (x *UpdateNamespaceResponse) GetFailoverVersion() int64 { + if x != nil { + return x.FailoverVersion + } + return 0 +} + +func (x *UpdateNamespaceResponse) GetIsGlobalNamespace() bool { + if x != nil { + return x.IsGlobalNamespace + } + return false +} + +// Deprecated. +type DeprecateNamespaceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + SecurityToken string `protobuf:"bytes,2,opt,name=security_token,json=securityToken,proto3" json:"security_token,omitempty"` +} + +func (x *DeprecateNamespaceRequest) Reset() { + *x = DeprecateNamespaceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeprecateNamespaceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeprecateNamespaceRequest) ProtoMessage() {} + +func (x *DeprecateNamespaceRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeprecateNamespaceRequest.ProtoReflect.Descriptor instead. +func (*DeprecateNamespaceRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{8} +} + +func (x *DeprecateNamespaceRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *DeprecateNamespaceRequest) GetSecurityToken() string { + if x != nil { + return x.SecurityToken + } + return "" +} + +// Deprecated. +type DeprecateNamespaceResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *DeprecateNamespaceResponse) Reset() { + *x = DeprecateNamespaceResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeprecateNamespaceResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeprecateNamespaceResponse) ProtoMessage() {} + +func (x *DeprecateNamespaceResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeprecateNamespaceResponse.ProtoReflect.Descriptor instead. +func (*DeprecateNamespaceResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{9} +} + +type StartWorkflowExecutionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + WorkflowId string `protobuf:"bytes,2,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + WorkflowType *v13.WorkflowType `protobuf:"bytes,3,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + TaskQueue *v14.TaskQueue `protobuf:"bytes,4,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + // Serialized arguments to the workflow. These are passed as arguments to the workflow function. + Input *v13.Payloads `protobuf:"bytes,5,opt,name=input,proto3" json:"input,omitempty"` + // Total workflow execution timeout including retries and continue as new. + WorkflowExecutionTimeout *durationpb.Duration `protobuf:"bytes,6,opt,name=workflow_execution_timeout,json=workflowExecutionTimeout,proto3" json:"workflow_execution_timeout,omitempty"` + // Timeout of a single workflow run. + WorkflowRunTimeout *durationpb.Duration `protobuf:"bytes,7,opt,name=workflow_run_timeout,json=workflowRunTimeout,proto3" json:"workflow_run_timeout,omitempty"` + // Timeout of a single workflow task. + WorkflowTaskTimeout *durationpb.Duration `protobuf:"bytes,8,opt,name=workflow_task_timeout,json=workflowTaskTimeout,proto3" json:"workflow_task_timeout,omitempty"` + // The identity of the client who initiated this request + Identity string `protobuf:"bytes,9,opt,name=identity,proto3" json:"identity,omitempty"` + // A unique identifier for this start request. Typically UUIDv4. + RequestId string `protobuf:"bytes,10,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` + // Defines whether to allow re-using the workflow id from a previously *closed* workflow. + // The default policy is WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE. + // + // See `workflow_id_conflict_policy` for handling a workflow id duplication with a *running* workflow. + WorkflowIdReusePolicy v11.WorkflowIdReusePolicy `protobuf:"varint,11,opt,name=workflow_id_reuse_policy,json=workflowIdReusePolicy,proto3,enum=temporal.api.enums.v1.WorkflowIdReusePolicy" json:"workflow_id_reuse_policy,omitempty"` + // Defines how to resolve a workflow id conflict with a *running* workflow. + // The default policy is WORKFLOW_ID_CONFLICT_POLICY_FAIL. + // + // See `workflow_id_reuse_policy` for handling a workflow id duplication with a *closed* workflow. + WorkflowIdConflictPolicy v11.WorkflowIdConflictPolicy `protobuf:"varint,22,opt,name=workflow_id_conflict_policy,json=workflowIdConflictPolicy,proto3,enum=temporal.api.enums.v1.WorkflowIdConflictPolicy" json:"workflow_id_conflict_policy,omitempty"` + // The retry policy for the workflow. Will never exceed `workflow_execution_timeout`. + RetryPolicy *v13.RetryPolicy `protobuf:"bytes,12,opt,name=retry_policy,json=retryPolicy,proto3" json:"retry_policy,omitempty"` + // See https://docs.temporal.io/docs/content/what-is-a-temporal-cron-job/ + CronSchedule string `protobuf:"bytes,13,opt,name=cron_schedule,json=cronSchedule,proto3" json:"cron_schedule,omitempty"` + Memo *v13.Memo `protobuf:"bytes,14,opt,name=memo,proto3" json:"memo,omitempty"` + SearchAttributes *v13.SearchAttributes `protobuf:"bytes,15,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` + Header *v13.Header `protobuf:"bytes,16,opt,name=header,proto3" json:"header,omitempty"` + // Request to get the first workflow task inline in the response bypassing matching service and worker polling. + // If set to `true` the caller is expected to have a worker available and capable of processing the task. + // The returned task will be marked as started and is expected to be completed by the specified + // `workflow_task_timeout`. + RequestEagerExecution bool `protobuf:"varint,17,opt,name=request_eager_execution,json=requestEagerExecution,proto3" json:"request_eager_execution,omitempty"` + // These values will be available as ContinuedFailure and LastCompletionResult in the + // WorkflowExecutionStarted event and through SDKs. The are currently only used by the + // server itself (for the schedules feature) and are not intended to be exposed in + // StartWorkflowExecution. + ContinuedFailure *v15.Failure `protobuf:"bytes,18,opt,name=continued_failure,json=continuedFailure,proto3" json:"continued_failure,omitempty"` + LastCompletionResult *v13.Payloads `protobuf:"bytes,19,opt,name=last_completion_result,json=lastCompletionResult,proto3" json:"last_completion_result,omitempty"` + // Time to wait before dispatching the first workflow task. Cannot be used with `cron_schedule`. + // If the workflow gets a signal before the delay, a workflow task will be dispatched and the rest + // of the delay will be ignored. + WorkflowStartDelay *durationpb.Duration `protobuf:"bytes,20,opt,name=workflow_start_delay,json=workflowStartDelay,proto3" json:"workflow_start_delay,omitempty"` + // Callbacks to be called by the server when this workflow reaches a terminal state. + // If the workflow continues-as-new, these callbacks will be carried over to the new execution. + // Callback addresses must be whitelisted in the server's dynamic configuration. + CompletionCallbacks []*v13.Callback `protobuf:"bytes,21,rep,name=completion_callbacks,json=completionCallbacks,proto3" json:"completion_callbacks,omitempty"` + // Metadata on the workflow if it is started. This is carried over to the WorkflowExecutionInfo + // for use by user interfaces to display the fixed as-of-start summary and details of the + // workflow. + UserMetadata *v16.UserMetadata `protobuf:"bytes,23,opt,name=user_metadata,json=userMetadata,proto3" json:"user_metadata,omitempty"` + // Links to be associated with the workflow. + Links []*v13.Link `protobuf:"bytes,24,rep,name=links,proto3" json:"links,omitempty"` +} + +func (x *StartWorkflowExecutionRequest) Reset() { + *x = StartWorkflowExecutionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartWorkflowExecutionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartWorkflowExecutionRequest) ProtoMessage() {} + +func (x *StartWorkflowExecutionRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartWorkflowExecutionRequest.ProtoReflect.Descriptor instead. +func (*StartWorkflowExecutionRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{10} +} + +func (x *StartWorkflowExecutionRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *StartWorkflowExecutionRequest) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *StartWorkflowExecutionRequest) GetWorkflowType() *v13.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetTaskQueue() *v14.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetInput() *v13.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetWorkflowExecutionTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowExecutionTimeout + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetWorkflowRunTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowRunTimeout + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetWorkflowTaskTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowTaskTimeout + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *StartWorkflowExecutionRequest) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +func (x *StartWorkflowExecutionRequest) GetWorkflowIdReusePolicy() v11.WorkflowIdReusePolicy { + if x != nil { + return x.WorkflowIdReusePolicy + } + return v11.WorkflowIdReusePolicy(0) +} + +func (x *StartWorkflowExecutionRequest) GetWorkflowIdConflictPolicy() v11.WorkflowIdConflictPolicy { + if x != nil { + return x.WorkflowIdConflictPolicy + } + return v11.WorkflowIdConflictPolicy(0) +} + +func (x *StartWorkflowExecutionRequest) GetRetryPolicy() *v13.RetryPolicy { + if x != nil { + return x.RetryPolicy + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetCronSchedule() string { + if x != nil { + return x.CronSchedule + } + return "" +} + +func (x *StartWorkflowExecutionRequest) GetMemo() *v13.Memo { + if x != nil { + return x.Memo + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetSearchAttributes() *v13.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetHeader() *v13.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetRequestEagerExecution() bool { + if x != nil { + return x.RequestEagerExecution + } + return false +} + +func (x *StartWorkflowExecutionRequest) GetContinuedFailure() *v15.Failure { + if x != nil { + return x.ContinuedFailure + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetLastCompletionResult() *v13.Payloads { + if x != nil { + return x.LastCompletionResult + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetWorkflowStartDelay() *durationpb.Duration { + if x != nil { + return x.WorkflowStartDelay + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetCompletionCallbacks() []*v13.Callback { + if x != nil { + return x.CompletionCallbacks + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetUserMetadata() *v16.UserMetadata { + if x != nil { + return x.UserMetadata + } + return nil +} + +func (x *StartWorkflowExecutionRequest) GetLinks() []*v13.Link { + if x != nil { + return x.Links + } + return nil +} + +type StartWorkflowExecutionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The run id of the workflow that was started - or used (via WorkflowIdConflictPolicy USE_EXISTING). + RunId string `protobuf:"bytes,1,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` + // If true, a new workflow was started. + Started bool `protobuf:"varint,3,opt,name=started,proto3" json:"started,omitempty"` + // When `request_eager_execution` is set on the `StartWorkflowExecutionRequest`, the server - if supported - will + // return the first workflow task to be eagerly executed. + // The caller is expected to have a worker available to process the task. + EagerWorkflowTask *PollWorkflowTaskQueueResponse `protobuf:"bytes,2,opt,name=eager_workflow_task,json=eagerWorkflowTask,proto3" json:"eager_workflow_task,omitempty"` +} + +func (x *StartWorkflowExecutionResponse) Reset() { + *x = StartWorkflowExecutionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartWorkflowExecutionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartWorkflowExecutionResponse) ProtoMessage() {} + +func (x *StartWorkflowExecutionResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartWorkflowExecutionResponse.ProtoReflect.Descriptor instead. +func (*StartWorkflowExecutionResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{11} +} + +func (x *StartWorkflowExecutionResponse) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +func (x *StartWorkflowExecutionResponse) GetStarted() bool { + if x != nil { + return x.Started + } + return false +} + +func (x *StartWorkflowExecutionResponse) GetEagerWorkflowTask() *PollWorkflowTaskQueueResponse { + if x != nil { + return x.EagerWorkflowTask + } + return nil +} + +type GetWorkflowExecutionHistoryRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + Execution *v13.WorkflowExecution `protobuf:"bytes,2,opt,name=execution,proto3" json:"execution,omitempty"` + MaximumPageSize int32 `protobuf:"varint,3,opt,name=maximum_page_size,json=maximumPageSize,proto3" json:"maximum_page_size,omitempty"` + // If a `GetWorkflowExecutionHistoryResponse` or a `PollWorkflowTaskQueueResponse` had one of + // these, it should be passed here to fetch the next page. + NextPageToken []byte `protobuf:"bytes,4,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` + // If set to true, the RPC call will not resolve until there is a new event which matches + // the `history_event_filter_type`, or a timeout is hit. + WaitNewEvent bool `protobuf:"varint,5,opt,name=wait_new_event,json=waitNewEvent,proto3" json:"wait_new_event,omitempty"` + // Filter returned events such that they match the specified filter type. + // Default: HISTORY_EVENT_FILTER_TYPE_ALL_EVENT. + HistoryEventFilterType v11.HistoryEventFilterType `protobuf:"varint,6,opt,name=history_event_filter_type,json=historyEventFilterType,proto3,enum=temporal.api.enums.v1.HistoryEventFilterType" json:"history_event_filter_type,omitempty"` + SkipArchival bool `protobuf:"varint,7,opt,name=skip_archival,json=skipArchival,proto3" json:"skip_archival,omitempty"` +} + +func (x *GetWorkflowExecutionHistoryRequest) Reset() { + *x = GetWorkflowExecutionHistoryRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetWorkflowExecutionHistoryRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetWorkflowExecutionHistoryRequest) ProtoMessage() {} + +func (x *GetWorkflowExecutionHistoryRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetWorkflowExecutionHistoryRequest.ProtoReflect.Descriptor instead. +func (*GetWorkflowExecutionHistoryRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{12} +} + +func (x *GetWorkflowExecutionHistoryRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *GetWorkflowExecutionHistoryRequest) GetExecution() *v13.WorkflowExecution { + if x != nil { + return x.Execution + } + return nil +} + +func (x *GetWorkflowExecutionHistoryRequest) GetMaximumPageSize() int32 { + if x != nil { + return x.MaximumPageSize + } + return 0 +} + +func (x *GetWorkflowExecutionHistoryRequest) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +func (x *GetWorkflowExecutionHistoryRequest) GetWaitNewEvent() bool { + if x != nil { + return x.WaitNewEvent + } + return false +} + +func (x *GetWorkflowExecutionHistoryRequest) GetHistoryEventFilterType() v11.HistoryEventFilterType { + if x != nil { + return x.HistoryEventFilterType + } + return v11.HistoryEventFilterType(0) +} + +func (x *GetWorkflowExecutionHistoryRequest) GetSkipArchival() bool { + if x != nil { + return x.SkipArchival + } + return false +} + +type GetWorkflowExecutionHistoryResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + History *v17.History `protobuf:"bytes,1,opt,name=history,proto3" json:"history,omitempty"` + // Raw history is an alternate representation of history that may be returned if configured on + // the frontend. This is not supported by all SDKs. Either this or `history` will be set. + RawHistory []*v13.DataBlob `protobuf:"bytes,2,rep,name=raw_history,json=rawHistory,proto3" json:"raw_history,omitempty"` + // Will be set if there are more history events than were included in this response + NextPageToken []byte `protobuf:"bytes,3,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` + Archived bool `protobuf:"varint,4,opt,name=archived,proto3" json:"archived,omitempty"` +} + +func (x *GetWorkflowExecutionHistoryResponse) Reset() { + *x = GetWorkflowExecutionHistoryResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetWorkflowExecutionHistoryResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetWorkflowExecutionHistoryResponse) ProtoMessage() {} + +func (x *GetWorkflowExecutionHistoryResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetWorkflowExecutionHistoryResponse.ProtoReflect.Descriptor instead. +func (*GetWorkflowExecutionHistoryResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{13} +} + +func (x *GetWorkflowExecutionHistoryResponse) GetHistory() *v17.History { + if x != nil { + return x.History + } + return nil +} + +func (x *GetWorkflowExecutionHistoryResponse) GetRawHistory() []*v13.DataBlob { + if x != nil { + return x.RawHistory + } + return nil +} + +func (x *GetWorkflowExecutionHistoryResponse) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +func (x *GetWorkflowExecutionHistoryResponse) GetArchived() bool { + if x != nil { + return x.Archived + } + return false +} + +type GetWorkflowExecutionHistoryReverseRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + Execution *v13.WorkflowExecution `protobuf:"bytes,2,opt,name=execution,proto3" json:"execution,omitempty"` + MaximumPageSize int32 `protobuf:"varint,3,opt,name=maximum_page_size,json=maximumPageSize,proto3" json:"maximum_page_size,omitempty"` + NextPageToken []byte `protobuf:"bytes,4,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *GetWorkflowExecutionHistoryReverseRequest) Reset() { + *x = GetWorkflowExecutionHistoryReverseRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetWorkflowExecutionHistoryReverseRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetWorkflowExecutionHistoryReverseRequest) ProtoMessage() {} + +func (x *GetWorkflowExecutionHistoryReverseRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetWorkflowExecutionHistoryReverseRequest.ProtoReflect.Descriptor instead. +func (*GetWorkflowExecutionHistoryReverseRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{14} +} + +func (x *GetWorkflowExecutionHistoryReverseRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *GetWorkflowExecutionHistoryReverseRequest) GetExecution() *v13.WorkflowExecution { + if x != nil { + return x.Execution + } + return nil +} + +func (x *GetWorkflowExecutionHistoryReverseRequest) GetMaximumPageSize() int32 { + if x != nil { + return x.MaximumPageSize + } + return 0 +} + +func (x *GetWorkflowExecutionHistoryReverseRequest) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +type GetWorkflowExecutionHistoryReverseResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + History *v17.History `protobuf:"bytes,1,opt,name=history,proto3" json:"history,omitempty"` + // Will be set if there are more history events than were included in this response + NextPageToken []byte `protobuf:"bytes,3,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *GetWorkflowExecutionHistoryReverseResponse) Reset() { + *x = GetWorkflowExecutionHistoryReverseResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetWorkflowExecutionHistoryReverseResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetWorkflowExecutionHistoryReverseResponse) ProtoMessage() {} + +func (x *GetWorkflowExecutionHistoryReverseResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[15] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetWorkflowExecutionHistoryReverseResponse.ProtoReflect.Descriptor instead. +func (*GetWorkflowExecutionHistoryReverseResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{15} +} + +func (x *GetWorkflowExecutionHistoryReverseResponse) GetHistory() *v17.History { + if x != nil { + return x.History + } + return nil +} + +func (x *GetWorkflowExecutionHistoryReverseResponse) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +type PollWorkflowTaskQueueRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + TaskQueue *v14.TaskQueue `protobuf:"bytes,2,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + // The identity of the worker/client who is polling this task queue + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` + // DEPRECATED since 1.21 - use `worker_version_capabilities` instead. + // Each worker process should provide an ID unique to the specific set of code it is running + // "checksum" in this field name isn't very accurate, it should be though of as an id. + BinaryChecksum string `protobuf:"bytes,4,opt,name=binary_checksum,json=binaryChecksum,proto3" json:"binary_checksum,omitempty"` + // Information about this worker's build identifier and if it is choosing to use the versioning + // feature. See the `WorkerVersionCapabilities` docstring for more. + WorkerVersionCapabilities *v13.WorkerVersionCapabilities `protobuf:"bytes,5,opt,name=worker_version_capabilities,json=workerVersionCapabilities,proto3" json:"worker_version_capabilities,omitempty"` +} + +func (x *PollWorkflowTaskQueueRequest) Reset() { + *x = PollWorkflowTaskQueueRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PollWorkflowTaskQueueRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PollWorkflowTaskQueueRequest) ProtoMessage() {} + +func (x *PollWorkflowTaskQueueRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[16] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PollWorkflowTaskQueueRequest.ProtoReflect.Descriptor instead. +func (*PollWorkflowTaskQueueRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{16} +} + +func (x *PollWorkflowTaskQueueRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *PollWorkflowTaskQueueRequest) GetTaskQueue() *v14.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *PollWorkflowTaskQueueRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *PollWorkflowTaskQueueRequest) GetBinaryChecksum() string { + if x != nil { + return x.BinaryChecksum + } + return "" +} + +func (x *PollWorkflowTaskQueueRequest) GetWorkerVersionCapabilities() *v13.WorkerVersionCapabilities { + if x != nil { + return x.WorkerVersionCapabilities + } + return nil +} + +type PollWorkflowTaskQueueResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // A unique identifier for this task + TaskToken []byte `protobuf:"bytes,1,opt,name=task_token,json=taskToken,proto3" json:"task_token,omitempty"` + WorkflowExecution *v13.WorkflowExecution `protobuf:"bytes,2,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + WorkflowType *v13.WorkflowType `protobuf:"bytes,3,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + // The last workflow task started event which was processed by some worker for this execution. + // Will be zero if no task has ever started. + PreviousStartedEventId int64 `protobuf:"varint,4,opt,name=previous_started_event_id,json=previousStartedEventId,proto3" json:"previous_started_event_id,omitempty"` + // The id of the most recent workflow task started event, which will have been generated as a + // result of this poll request being served. Will be zero if the task + // does not contain any events which would advance history (no new WFT started). + // Currently this can happen for queries. + StartedEventId int64 `protobuf:"varint,5,opt,name=started_event_id,json=startedEventId,proto3" json:"started_event_id,omitempty"` + // Starting at 1, the number of attempts to complete this task by any worker. + Attempt int32 `protobuf:"varint,6,opt,name=attempt,proto3" json:"attempt,omitempty"` + // A hint that there are more tasks already present in this task queue + // partition. Can be used to prioritize draining a sticky queue. + // + // Specifically, the returned number is the number of tasks remaining in + // the in-memory buffer for this partition, which is currently capped at + // 1000. Because sticky queues only have one partition, this number is + // more useful when draining them. Normal queues, typically having more than one + // partition, will return a number representing only some portion of the + // overall backlog. Subsequent RPCs may not hit the same partition as + // this call. + BacklogCountHint int64 `protobuf:"varint,7,opt,name=backlog_count_hint,json=backlogCountHint,proto3" json:"backlog_count_hint,omitempty"` + // The history for this workflow, which will either be complete or partial. Partial histories + // are sent to workers who have signaled that they are using a sticky queue when completing + // a workflow task. + History *v17.History `protobuf:"bytes,8,opt,name=history,proto3" json:"history,omitempty"` + // Will be set if there are more history events than were included in this response. Such events + // should be fetched via `GetWorkflowExecutionHistory`. + NextPageToken []byte `protobuf:"bytes,9,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` + // Legacy queries appear in this field. The query must be responded to via + // `RespondQueryTaskCompleted`. If the workflow is already closed (queries are permitted on + // closed workflows) then the `history` field will be populated with the entire history. It + // may also be populated if this task originates on a non-sticky queue. + Query *v18.WorkflowQuery `protobuf:"bytes,10,opt,name=query,proto3" json:"query,omitempty"` + // The task queue this task originated from, which will always be the original non-sticky name + // for the queue, even if this response came from polling a sticky queue. + WorkflowExecutionTaskQueue *v14.TaskQueue `protobuf:"bytes,11,opt,name=workflow_execution_task_queue,json=workflowExecutionTaskQueue,proto3" json:"workflow_execution_task_queue,omitempty"` + // When this task was scheduled by the server + ScheduledTime *timestamppb.Timestamp `protobuf:"bytes,12,opt,name=scheduled_time,json=scheduledTime,proto3" json:"scheduled_time,omitempty"` + // When the current workflow task started event was generated, meaning the current attempt. + StartedTime *timestamppb.Timestamp `protobuf:"bytes,13,opt,name=started_time,json=startedTime,proto3" json:"started_time,omitempty"` + // Queries that should be executed after applying the history in this task. Responses should be + // attached to `RespondWorkflowTaskCompletedRequest::query_results` + Queries map[string]*v18.WorkflowQuery `protobuf:"bytes,14,rep,name=queries,proto3" json:"queries,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // Protocol messages piggybacking on a WFT as a transport + Messages []*v19.Message `protobuf:"bytes,15,rep,name=messages,proto3" json:"messages,omitempty"` +} + +func (x *PollWorkflowTaskQueueResponse) Reset() { + *x = PollWorkflowTaskQueueResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PollWorkflowTaskQueueResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PollWorkflowTaskQueueResponse) ProtoMessage() {} + +func (x *PollWorkflowTaskQueueResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[17] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PollWorkflowTaskQueueResponse.ProtoReflect.Descriptor instead. +func (*PollWorkflowTaskQueueResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{17} +} + +func (x *PollWorkflowTaskQueueResponse) GetTaskToken() []byte { + if x != nil { + return x.TaskToken + } + return nil +} + +func (x *PollWorkflowTaskQueueResponse) GetWorkflowExecution() *v13.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *PollWorkflowTaskQueueResponse) GetWorkflowType() *v13.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *PollWorkflowTaskQueueResponse) GetPreviousStartedEventId() int64 { + if x != nil { + return x.PreviousStartedEventId + } + return 0 +} + +func (x *PollWorkflowTaskQueueResponse) GetStartedEventId() int64 { + if x != nil { + return x.StartedEventId + } + return 0 +} + +func (x *PollWorkflowTaskQueueResponse) GetAttempt() int32 { + if x != nil { + return x.Attempt + } + return 0 +} + +func (x *PollWorkflowTaskQueueResponse) GetBacklogCountHint() int64 { + if x != nil { + return x.BacklogCountHint + } + return 0 +} + +func (x *PollWorkflowTaskQueueResponse) GetHistory() *v17.History { + if x != nil { + return x.History + } + return nil +} + +func (x *PollWorkflowTaskQueueResponse) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +func (x *PollWorkflowTaskQueueResponse) GetQuery() *v18.WorkflowQuery { + if x != nil { + return x.Query + } + return nil +} + +func (x *PollWorkflowTaskQueueResponse) GetWorkflowExecutionTaskQueue() *v14.TaskQueue { + if x != nil { + return x.WorkflowExecutionTaskQueue + } + return nil +} + +func (x *PollWorkflowTaskQueueResponse) GetScheduledTime() *timestamppb.Timestamp { + if x != nil { + return x.ScheduledTime + } + return nil +} + +func (x *PollWorkflowTaskQueueResponse) GetStartedTime() *timestamppb.Timestamp { + if x != nil { + return x.StartedTime + } + return nil +} + +func (x *PollWorkflowTaskQueueResponse) GetQueries() map[string]*v18.WorkflowQuery { + if x != nil { + return x.Queries + } + return nil +} + +func (x *PollWorkflowTaskQueueResponse) GetMessages() []*v19.Message { + if x != nil { + return x.Messages + } + return nil +} + +type RespondWorkflowTaskCompletedRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The task token as received in `PollWorkflowTaskQueueResponse` + TaskToken []byte `protobuf:"bytes,1,opt,name=task_token,json=taskToken,proto3" json:"task_token,omitempty"` + // A list of commands generated when driving the workflow code in response to the new task + Commands []*v110.Command `protobuf:"bytes,2,rep,name=commands,proto3" json:"commands,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` + // May be set by workers to indicate that the worker desires future tasks to be provided with + // incremental history on a sticky queue. + StickyAttributes *v14.StickyExecutionAttributes `protobuf:"bytes,4,opt,name=sticky_attributes,json=stickyAttributes,proto3" json:"sticky_attributes,omitempty"` + // If set, the worker wishes to immediately receive the next workflow task as a response to + // this completion. This can save on polling round-trips. + ReturnNewWorkflowTask bool `protobuf:"varint,5,opt,name=return_new_workflow_task,json=returnNewWorkflowTask,proto3" json:"return_new_workflow_task,omitempty"` + // Can be used to *force* creation of a new workflow task, even if no commands have resolved or + // one would not otherwise have been generated. This is used when the worker knows it is doing + // something useful, but cannot complete it within the workflow task timeout. Local activities + // which run for longer than the task timeout being the prime example. + ForceCreateNewWorkflowTask bool `protobuf:"varint,6,opt,name=force_create_new_workflow_task,json=forceCreateNewWorkflowTask,proto3" json:"force_create_new_workflow_task,omitempty"` + // DEPRECATED since 1.21 - use `worker_version_stamp` instead. + // Worker process' unique binary id + BinaryChecksum string `protobuf:"bytes,7,opt,name=binary_checksum,json=binaryChecksum,proto3" json:"binary_checksum,omitempty"` + // Responses to the `queries` field in the task being responded to + QueryResults map[string]*v18.WorkflowQueryResult `protobuf:"bytes,8,rep,name=query_results,json=queryResults,proto3" json:"query_results,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Namespace string `protobuf:"bytes,9,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Version info of the worker who processed this task. This message's `build_id` field should + // always be set by SDKs. Workers opting into versioning will also set the `use_versioning` + // field to true. See message docstrings for more. + WorkerVersionStamp *v13.WorkerVersionStamp `protobuf:"bytes,10,opt,name=worker_version_stamp,json=workerVersionStamp,proto3" json:"worker_version_stamp,omitempty"` + // Protocol messages piggybacking on a WFT as a transport + Messages []*v19.Message `protobuf:"bytes,11,rep,name=messages,proto3" json:"messages,omitempty"` + // Data the SDK wishes to record for itself, but server need not interpret, and does not + // directly impact workflow state. + SdkMetadata *v16.WorkflowTaskCompletedMetadata `protobuf:"bytes,12,opt,name=sdk_metadata,json=sdkMetadata,proto3" json:"sdk_metadata,omitempty"` + // Local usage data collected for metering + MeteringMetadata *v13.MeteringMetadata `protobuf:"bytes,13,opt,name=metering_metadata,json=meteringMetadata,proto3" json:"metering_metadata,omitempty"` +} + +func (x *RespondWorkflowTaskCompletedRequest) Reset() { + *x = RespondWorkflowTaskCompletedRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondWorkflowTaskCompletedRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondWorkflowTaskCompletedRequest) ProtoMessage() {} + +func (x *RespondWorkflowTaskCompletedRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[18] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondWorkflowTaskCompletedRequest.ProtoReflect.Descriptor instead. +func (*RespondWorkflowTaskCompletedRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{18} +} + +func (x *RespondWorkflowTaskCompletedRequest) GetTaskToken() []byte { + if x != nil { + return x.TaskToken + } + return nil +} + +func (x *RespondWorkflowTaskCompletedRequest) GetCommands() []*v110.Command { + if x != nil { + return x.Commands + } + return nil +} + +func (x *RespondWorkflowTaskCompletedRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *RespondWorkflowTaskCompletedRequest) GetStickyAttributes() *v14.StickyExecutionAttributes { + if x != nil { + return x.StickyAttributes + } + return nil +} + +func (x *RespondWorkflowTaskCompletedRequest) GetReturnNewWorkflowTask() bool { + if x != nil { + return x.ReturnNewWorkflowTask + } + return false +} + +func (x *RespondWorkflowTaskCompletedRequest) GetForceCreateNewWorkflowTask() bool { + if x != nil { + return x.ForceCreateNewWorkflowTask + } + return false +} + +func (x *RespondWorkflowTaskCompletedRequest) GetBinaryChecksum() string { + if x != nil { + return x.BinaryChecksum + } + return "" +} + +func (x *RespondWorkflowTaskCompletedRequest) GetQueryResults() map[string]*v18.WorkflowQueryResult { + if x != nil { + return x.QueryResults + } + return nil +} + +func (x *RespondWorkflowTaskCompletedRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RespondWorkflowTaskCompletedRequest) GetWorkerVersionStamp() *v13.WorkerVersionStamp { + if x != nil { + return x.WorkerVersionStamp + } + return nil +} + +func (x *RespondWorkflowTaskCompletedRequest) GetMessages() []*v19.Message { + if x != nil { + return x.Messages + } + return nil +} + +func (x *RespondWorkflowTaskCompletedRequest) GetSdkMetadata() *v16.WorkflowTaskCompletedMetadata { + if x != nil { + return x.SdkMetadata + } + return nil +} + +func (x *RespondWorkflowTaskCompletedRequest) GetMeteringMetadata() *v13.MeteringMetadata { + if x != nil { + return x.MeteringMetadata + } + return nil +} + +type RespondWorkflowTaskCompletedResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // See `RespondWorkflowTaskCompletedResponse::return_new_workflow_task` + WorkflowTask *PollWorkflowTaskQueueResponse `protobuf:"bytes,1,opt,name=workflow_task,json=workflowTask,proto3" json:"workflow_task,omitempty"` + // See `ScheduleActivityTaskCommandAttributes::request_eager_execution` + ActivityTasks []*PollActivityTaskQueueResponse `protobuf:"bytes,2,rep,name=activity_tasks,json=activityTasks,proto3" json:"activity_tasks,omitempty"` + // If non zero, indicates the server has discarded the workflow task that was being responded to. + // Will be the event ID of the last workflow task started event in the history before the new workflow task. + // Server is only expected to discard a workflow task if it could not have modified the workflow state. + ResetHistoryEventId int64 `protobuf:"varint,3,opt,name=reset_history_event_id,json=resetHistoryEventId,proto3" json:"reset_history_event_id,omitempty"` +} + +func (x *RespondWorkflowTaskCompletedResponse) Reset() { + *x = RespondWorkflowTaskCompletedResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondWorkflowTaskCompletedResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondWorkflowTaskCompletedResponse) ProtoMessage() {} + +func (x *RespondWorkflowTaskCompletedResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[19] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondWorkflowTaskCompletedResponse.ProtoReflect.Descriptor instead. +func (*RespondWorkflowTaskCompletedResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{19} +} + +func (x *RespondWorkflowTaskCompletedResponse) GetWorkflowTask() *PollWorkflowTaskQueueResponse { + if x != nil { + return x.WorkflowTask + } + return nil +} + +func (x *RespondWorkflowTaskCompletedResponse) GetActivityTasks() []*PollActivityTaskQueueResponse { + if x != nil { + return x.ActivityTasks + } + return nil +} + +func (x *RespondWorkflowTaskCompletedResponse) GetResetHistoryEventId() int64 { + if x != nil { + return x.ResetHistoryEventId + } + return 0 +} + +type RespondWorkflowTaskFailedRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The task token as received in `PollWorkflowTaskQueueResponse` + TaskToken []byte `protobuf:"bytes,1,opt,name=task_token,json=taskToken,proto3" json:"task_token,omitempty"` + // Why did the task fail? It's important to note that many of the variants in this enum cannot + // apply to worker responses. See the type's doc for more. + Cause v11.WorkflowTaskFailedCause `protobuf:"varint,2,opt,name=cause,proto3,enum=temporal.api.enums.v1.WorkflowTaskFailedCause" json:"cause,omitempty"` + // Failure details + Failure *v15.Failure `protobuf:"bytes,3,opt,name=failure,proto3" json:"failure,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,4,opt,name=identity,proto3" json:"identity,omitempty"` + // DEPRECATED since 1.21 - use `worker_version_stamp` instead. + // Worker process' unique binary id + BinaryChecksum string `protobuf:"bytes,5,opt,name=binary_checksum,json=binaryChecksum,proto3" json:"binary_checksum,omitempty"` + Namespace string `protobuf:"bytes,6,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Protocol messages piggybacking on a WFT as a transport + Messages []*v19.Message `protobuf:"bytes,7,rep,name=messages,proto3" json:"messages,omitempty"` + // Version info of the worker who processed this task. This message's `build_id` field should + // always be set by SDKs. Workers opting into versioning will also set the `use_versioning` + // field to true. See message docstrings for more. + WorkerVersion *v13.WorkerVersionStamp `protobuf:"bytes,8,opt,name=worker_version,json=workerVersion,proto3" json:"worker_version,omitempty"` +} + +func (x *RespondWorkflowTaskFailedRequest) Reset() { + *x = RespondWorkflowTaskFailedRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondWorkflowTaskFailedRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondWorkflowTaskFailedRequest) ProtoMessage() {} + +func (x *RespondWorkflowTaskFailedRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[20] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondWorkflowTaskFailedRequest.ProtoReflect.Descriptor instead. +func (*RespondWorkflowTaskFailedRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{20} +} + +func (x *RespondWorkflowTaskFailedRequest) GetTaskToken() []byte { + if x != nil { + return x.TaskToken + } + return nil +} + +func (x *RespondWorkflowTaskFailedRequest) GetCause() v11.WorkflowTaskFailedCause { + if x != nil { + return x.Cause + } + return v11.WorkflowTaskFailedCause(0) +} + +func (x *RespondWorkflowTaskFailedRequest) GetFailure() *v15.Failure { + if x != nil { + return x.Failure + } + return nil +} + +func (x *RespondWorkflowTaskFailedRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *RespondWorkflowTaskFailedRequest) GetBinaryChecksum() string { + if x != nil { + return x.BinaryChecksum + } + return "" +} + +func (x *RespondWorkflowTaskFailedRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RespondWorkflowTaskFailedRequest) GetMessages() []*v19.Message { + if x != nil { + return x.Messages + } + return nil +} + +func (x *RespondWorkflowTaskFailedRequest) GetWorkerVersion() *v13.WorkerVersionStamp { + if x != nil { + return x.WorkerVersion + } + return nil +} + +type RespondWorkflowTaskFailedResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RespondWorkflowTaskFailedResponse) Reset() { + *x = RespondWorkflowTaskFailedResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondWorkflowTaskFailedResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondWorkflowTaskFailedResponse) ProtoMessage() {} + +func (x *RespondWorkflowTaskFailedResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[21] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondWorkflowTaskFailedResponse.ProtoReflect.Descriptor instead. +func (*RespondWorkflowTaskFailedResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{21} +} + +type PollActivityTaskQueueRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + TaskQueue *v14.TaskQueue `protobuf:"bytes,2,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` + TaskQueueMetadata *v14.TaskQueueMetadata `protobuf:"bytes,4,opt,name=task_queue_metadata,json=taskQueueMetadata,proto3" json:"task_queue_metadata,omitempty"` + // Information about this worker's build identifier and if it is choosing to use the versioning + // feature. See the `WorkerVersionCapabilities` docstring for more. + WorkerVersionCapabilities *v13.WorkerVersionCapabilities `protobuf:"bytes,5,opt,name=worker_version_capabilities,json=workerVersionCapabilities,proto3" json:"worker_version_capabilities,omitempty"` +} + +func (x *PollActivityTaskQueueRequest) Reset() { + *x = PollActivityTaskQueueRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PollActivityTaskQueueRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PollActivityTaskQueueRequest) ProtoMessage() {} + +func (x *PollActivityTaskQueueRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[22] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PollActivityTaskQueueRequest.ProtoReflect.Descriptor instead. +func (*PollActivityTaskQueueRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{22} +} + +func (x *PollActivityTaskQueueRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *PollActivityTaskQueueRequest) GetTaskQueue() *v14.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *PollActivityTaskQueueRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *PollActivityTaskQueueRequest) GetTaskQueueMetadata() *v14.TaskQueueMetadata { + if x != nil { + return x.TaskQueueMetadata + } + return nil +} + +func (x *PollActivityTaskQueueRequest) GetWorkerVersionCapabilities() *v13.WorkerVersionCapabilities { + if x != nil { + return x.WorkerVersionCapabilities + } + return nil +} + +type PollActivityTaskQueueResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // A unique identifier for this task + TaskToken []byte `protobuf:"bytes,1,opt,name=task_token,json=taskToken,proto3" json:"task_token,omitempty"` + // The namespace the workflow which requested this activity lives in + WorkflowNamespace string `protobuf:"bytes,2,opt,name=workflow_namespace,json=workflowNamespace,proto3" json:"workflow_namespace,omitempty"` + // Type of the requesting workflow + WorkflowType *v13.WorkflowType `protobuf:"bytes,3,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + // Execution info of the requesting workflow + WorkflowExecution *v13.WorkflowExecution `protobuf:"bytes,4,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + ActivityType *v13.ActivityType `protobuf:"bytes,5,opt,name=activity_type,json=activityType,proto3" json:"activity_type,omitempty"` + // The autogenerated or user specified identifier of this activity. Can be used to complete the + // activity via `RespondActivityTaskCompletedById`. May be re-used as long as the last usage + // has resolved, but unique IDs for every activity invocation is a good idea. + ActivityId string `protobuf:"bytes,6,opt,name=activity_id,json=activityId,proto3" json:"activity_id,omitempty"` + // Headers specified by the scheduling workflow. Commonly used to propagate contextual info + // from the workflow to its activities. For example, tracing contexts. + Header *v13.Header `protobuf:"bytes,7,opt,name=header,proto3" json:"header,omitempty"` + // Arguments to the activity invocation + Input *v13.Payloads `protobuf:"bytes,8,opt,name=input,proto3" json:"input,omitempty"` + // Details of the last heartbeat that was recorded for this activity as of the time this task + // was delivered. + HeartbeatDetails *v13.Payloads `protobuf:"bytes,9,opt,name=heartbeat_details,json=heartbeatDetails,proto3" json:"heartbeat_details,omitempty"` + // When was this task first scheduled + ScheduledTime *timestamppb.Timestamp `protobuf:"bytes,10,opt,name=scheduled_time,json=scheduledTime,proto3" json:"scheduled_time,omitempty"` + // When was this task attempt scheduled + CurrentAttemptScheduledTime *timestamppb.Timestamp `protobuf:"bytes,11,opt,name=current_attempt_scheduled_time,json=currentAttemptScheduledTime,proto3" json:"current_attempt_scheduled_time,omitempty"` + // When was this task started (this attempt) + StartedTime *timestamppb.Timestamp `protobuf:"bytes,12,opt,name=started_time,json=startedTime,proto3" json:"started_time,omitempty"` + // Starting at 1, the number of attempts to perform this activity + Attempt int32 `protobuf:"varint,13,opt,name=attempt,proto3" json:"attempt,omitempty"` + // First scheduled -> final result reported timeout + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + ScheduleToCloseTimeout *durationpb.Duration `protobuf:"bytes,14,opt,name=schedule_to_close_timeout,json=scheduleToCloseTimeout,proto3" json:"schedule_to_close_timeout,omitempty"` + // Current attempt start -> final result reported timeout + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: "to" is used to indicate interval. --) + StartToCloseTimeout *durationpb.Duration `protobuf:"bytes,15,opt,name=start_to_close_timeout,json=startToCloseTimeout,proto3" json:"start_to_close_timeout,omitempty"` + // Window within which the activity must report a heartbeat, or be timed out. + HeartbeatTimeout *durationpb.Duration `protobuf:"bytes,16,opt,name=heartbeat_timeout,json=heartbeatTimeout,proto3" json:"heartbeat_timeout,omitempty"` + // This is the retry policy the service uses which may be different from the one provided + // (or not) during activity scheduling. The service can override the provided one if some + // values are not specified or exceed configured system limits. + RetryPolicy *v13.RetryPolicy `protobuf:"bytes,17,opt,name=retry_policy,json=retryPolicy,proto3" json:"retry_policy,omitempty"` +} + +func (x *PollActivityTaskQueueResponse) Reset() { + *x = PollActivityTaskQueueResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PollActivityTaskQueueResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PollActivityTaskQueueResponse) ProtoMessage() {} + +func (x *PollActivityTaskQueueResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[23] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PollActivityTaskQueueResponse.ProtoReflect.Descriptor instead. +func (*PollActivityTaskQueueResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{23} +} + +func (x *PollActivityTaskQueueResponse) GetTaskToken() []byte { + if x != nil { + return x.TaskToken + } + return nil +} + +func (x *PollActivityTaskQueueResponse) GetWorkflowNamespace() string { + if x != nil { + return x.WorkflowNamespace + } + return "" +} + +func (x *PollActivityTaskQueueResponse) GetWorkflowType() *v13.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *PollActivityTaskQueueResponse) GetWorkflowExecution() *v13.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *PollActivityTaskQueueResponse) GetActivityType() *v13.ActivityType { + if x != nil { + return x.ActivityType + } + return nil +} + +func (x *PollActivityTaskQueueResponse) GetActivityId() string { + if x != nil { + return x.ActivityId + } + return "" +} + +func (x *PollActivityTaskQueueResponse) GetHeader() *v13.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *PollActivityTaskQueueResponse) GetInput() *v13.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *PollActivityTaskQueueResponse) GetHeartbeatDetails() *v13.Payloads { + if x != nil { + return x.HeartbeatDetails + } + return nil +} + +func (x *PollActivityTaskQueueResponse) GetScheduledTime() *timestamppb.Timestamp { + if x != nil { + return x.ScheduledTime + } + return nil +} + +func (x *PollActivityTaskQueueResponse) GetCurrentAttemptScheduledTime() *timestamppb.Timestamp { + if x != nil { + return x.CurrentAttemptScheduledTime + } + return nil +} + +func (x *PollActivityTaskQueueResponse) GetStartedTime() *timestamppb.Timestamp { + if x != nil { + return x.StartedTime + } + return nil +} + +func (x *PollActivityTaskQueueResponse) GetAttempt() int32 { + if x != nil { + return x.Attempt + } + return 0 +} + +func (x *PollActivityTaskQueueResponse) GetScheduleToCloseTimeout() *durationpb.Duration { + if x != nil { + return x.ScheduleToCloseTimeout + } + return nil +} + +func (x *PollActivityTaskQueueResponse) GetStartToCloseTimeout() *durationpb.Duration { + if x != nil { + return x.StartToCloseTimeout + } + return nil +} + +func (x *PollActivityTaskQueueResponse) GetHeartbeatTimeout() *durationpb.Duration { + if x != nil { + return x.HeartbeatTimeout + } + return nil +} + +func (x *PollActivityTaskQueueResponse) GetRetryPolicy() *v13.RetryPolicy { + if x != nil { + return x.RetryPolicy + } + return nil +} + +type RecordActivityTaskHeartbeatRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The task token as received in `PollActivityTaskQueueResponse` + TaskToken []byte `protobuf:"bytes,1,opt,name=task_token,json=taskToken,proto3" json:"task_token,omitempty"` + // Arbitrary data, of which the most recent call is kept, to store for this activity + Details *v13.Payloads `protobuf:"bytes,2,opt,name=details,proto3" json:"details,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` + Namespace string `protobuf:"bytes,4,opt,name=namespace,proto3" json:"namespace,omitempty"` +} + +func (x *RecordActivityTaskHeartbeatRequest) Reset() { + *x = RecordActivityTaskHeartbeatRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RecordActivityTaskHeartbeatRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RecordActivityTaskHeartbeatRequest) ProtoMessage() {} + +func (x *RecordActivityTaskHeartbeatRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[24] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RecordActivityTaskHeartbeatRequest.ProtoReflect.Descriptor instead. +func (*RecordActivityTaskHeartbeatRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{24} +} + +func (x *RecordActivityTaskHeartbeatRequest) GetTaskToken() []byte { + if x != nil { + return x.TaskToken + } + return nil +} + +func (x *RecordActivityTaskHeartbeatRequest) GetDetails() *v13.Payloads { + if x != nil { + return x.Details + } + return nil +} + +func (x *RecordActivityTaskHeartbeatRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *RecordActivityTaskHeartbeatRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +type RecordActivityTaskHeartbeatResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Will be set to true if the activity has been asked to cancel itself. The SDK should then + // notify the activity of cancellation if it is still running. + CancelRequested bool `protobuf:"varint,1,opt,name=cancel_requested,json=cancelRequested,proto3" json:"cancel_requested,omitempty"` +} + +func (x *RecordActivityTaskHeartbeatResponse) Reset() { + *x = RecordActivityTaskHeartbeatResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RecordActivityTaskHeartbeatResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RecordActivityTaskHeartbeatResponse) ProtoMessage() {} + +func (x *RecordActivityTaskHeartbeatResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[25] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RecordActivityTaskHeartbeatResponse.ProtoReflect.Descriptor instead. +func (*RecordActivityTaskHeartbeatResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{25} +} + +func (x *RecordActivityTaskHeartbeatResponse) GetCancelRequested() bool { + if x != nil { + return x.CancelRequested + } + return false +} + +type RecordActivityTaskHeartbeatByIdRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace of the workflow which scheduled this activity + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Id of the workflow which scheduled this activity + WorkflowId string `protobuf:"bytes,2,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + // Run Id of the workflow which scheduled this activity + RunId string `protobuf:"bytes,3,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` + // Id of the activity we're heartbeating + ActivityId string `protobuf:"bytes,4,opt,name=activity_id,json=activityId,proto3" json:"activity_id,omitempty"` + // Arbitrary data, of which the most recent call is kept, to store for this activity + Details *v13.Payloads `protobuf:"bytes,5,opt,name=details,proto3" json:"details,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,6,opt,name=identity,proto3" json:"identity,omitempty"` +} + +func (x *RecordActivityTaskHeartbeatByIdRequest) Reset() { + *x = RecordActivityTaskHeartbeatByIdRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RecordActivityTaskHeartbeatByIdRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RecordActivityTaskHeartbeatByIdRequest) ProtoMessage() {} + +func (x *RecordActivityTaskHeartbeatByIdRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[26] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RecordActivityTaskHeartbeatByIdRequest.ProtoReflect.Descriptor instead. +func (*RecordActivityTaskHeartbeatByIdRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{26} +} + +func (x *RecordActivityTaskHeartbeatByIdRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RecordActivityTaskHeartbeatByIdRequest) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *RecordActivityTaskHeartbeatByIdRequest) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +func (x *RecordActivityTaskHeartbeatByIdRequest) GetActivityId() string { + if x != nil { + return x.ActivityId + } + return "" +} + +func (x *RecordActivityTaskHeartbeatByIdRequest) GetDetails() *v13.Payloads { + if x != nil { + return x.Details + } + return nil +} + +func (x *RecordActivityTaskHeartbeatByIdRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +type RecordActivityTaskHeartbeatByIdResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Will be set to true if the activity has been asked to cancel itself. The SDK should then + // notify the activity of cancellation if it is still running. + CancelRequested bool `protobuf:"varint,1,opt,name=cancel_requested,json=cancelRequested,proto3" json:"cancel_requested,omitempty"` +} + +func (x *RecordActivityTaskHeartbeatByIdResponse) Reset() { + *x = RecordActivityTaskHeartbeatByIdResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RecordActivityTaskHeartbeatByIdResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RecordActivityTaskHeartbeatByIdResponse) ProtoMessage() {} + +func (x *RecordActivityTaskHeartbeatByIdResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[27] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RecordActivityTaskHeartbeatByIdResponse.ProtoReflect.Descriptor instead. +func (*RecordActivityTaskHeartbeatByIdResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{27} +} + +func (x *RecordActivityTaskHeartbeatByIdResponse) GetCancelRequested() bool { + if x != nil { + return x.CancelRequested + } + return false +} + +type RespondActivityTaskCompletedRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The task token as received in `PollActivityTaskQueueResponse` + TaskToken []byte `protobuf:"bytes,1,opt,name=task_token,json=taskToken,proto3" json:"task_token,omitempty"` + // The result of successfully executing the activity + Result *v13.Payloads `protobuf:"bytes,2,opt,name=result,proto3" json:"result,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` + Namespace string `protobuf:"bytes,4,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Version info of the worker who processed this task. This message's `build_id` field should + // always be set by SDKs. Workers opting into versioning will also set the `use_versioning` + // field to true. See message docstrings for more. + WorkerVersion *v13.WorkerVersionStamp `protobuf:"bytes,5,opt,name=worker_version,json=workerVersion,proto3" json:"worker_version,omitempty"` +} + +func (x *RespondActivityTaskCompletedRequest) Reset() { + *x = RespondActivityTaskCompletedRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondActivityTaskCompletedRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondActivityTaskCompletedRequest) ProtoMessage() {} + +func (x *RespondActivityTaskCompletedRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[28] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondActivityTaskCompletedRequest.ProtoReflect.Descriptor instead. +func (*RespondActivityTaskCompletedRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{28} +} + +func (x *RespondActivityTaskCompletedRequest) GetTaskToken() []byte { + if x != nil { + return x.TaskToken + } + return nil +} + +func (x *RespondActivityTaskCompletedRequest) GetResult() *v13.Payloads { + if x != nil { + return x.Result + } + return nil +} + +func (x *RespondActivityTaskCompletedRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *RespondActivityTaskCompletedRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RespondActivityTaskCompletedRequest) GetWorkerVersion() *v13.WorkerVersionStamp { + if x != nil { + return x.WorkerVersion + } + return nil +} + +type RespondActivityTaskCompletedResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RespondActivityTaskCompletedResponse) Reset() { + *x = RespondActivityTaskCompletedResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondActivityTaskCompletedResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondActivityTaskCompletedResponse) ProtoMessage() {} + +func (x *RespondActivityTaskCompletedResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[29] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondActivityTaskCompletedResponse.ProtoReflect.Descriptor instead. +func (*RespondActivityTaskCompletedResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{29} +} + +type RespondActivityTaskCompletedByIdRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace of the workflow which scheduled this activity + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Id of the workflow which scheduled this activity + WorkflowId string `protobuf:"bytes,2,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + // Run Id of the workflow which scheduled this activity + RunId string `protobuf:"bytes,3,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` + // Id of the activity to complete + ActivityId string `protobuf:"bytes,4,opt,name=activity_id,json=activityId,proto3" json:"activity_id,omitempty"` + // The serialized result of activity execution + Result *v13.Payloads `protobuf:"bytes,5,opt,name=result,proto3" json:"result,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,6,opt,name=identity,proto3" json:"identity,omitempty"` +} + +func (x *RespondActivityTaskCompletedByIdRequest) Reset() { + *x = RespondActivityTaskCompletedByIdRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondActivityTaskCompletedByIdRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondActivityTaskCompletedByIdRequest) ProtoMessage() {} + +func (x *RespondActivityTaskCompletedByIdRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[30] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondActivityTaskCompletedByIdRequest.ProtoReflect.Descriptor instead. +func (*RespondActivityTaskCompletedByIdRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{30} +} + +func (x *RespondActivityTaskCompletedByIdRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RespondActivityTaskCompletedByIdRequest) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *RespondActivityTaskCompletedByIdRequest) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +func (x *RespondActivityTaskCompletedByIdRequest) GetActivityId() string { + if x != nil { + return x.ActivityId + } + return "" +} + +func (x *RespondActivityTaskCompletedByIdRequest) GetResult() *v13.Payloads { + if x != nil { + return x.Result + } + return nil +} + +func (x *RespondActivityTaskCompletedByIdRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +type RespondActivityTaskCompletedByIdResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RespondActivityTaskCompletedByIdResponse) Reset() { + *x = RespondActivityTaskCompletedByIdResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondActivityTaskCompletedByIdResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondActivityTaskCompletedByIdResponse) ProtoMessage() {} + +func (x *RespondActivityTaskCompletedByIdResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[31] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondActivityTaskCompletedByIdResponse.ProtoReflect.Descriptor instead. +func (*RespondActivityTaskCompletedByIdResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{31} +} + +type RespondActivityTaskFailedRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The task token as received in `PollActivityTaskQueueResponse` + TaskToken []byte `protobuf:"bytes,1,opt,name=task_token,json=taskToken,proto3" json:"task_token,omitempty"` + // Detailed failure information + Failure *v15.Failure `protobuf:"bytes,2,opt,name=failure,proto3" json:"failure,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` + Namespace string `protobuf:"bytes,4,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Additional details to be stored as last activity heartbeat + LastHeartbeatDetails *v13.Payloads `protobuf:"bytes,5,opt,name=last_heartbeat_details,json=lastHeartbeatDetails,proto3" json:"last_heartbeat_details,omitempty"` + // Version info of the worker who processed this task. This message's `build_id` field should + // always be set by SDKs. Workers opting into versioning will also set the `use_versioning` + // field to true. See message docstrings for more. + WorkerVersion *v13.WorkerVersionStamp `protobuf:"bytes,6,opt,name=worker_version,json=workerVersion,proto3" json:"worker_version,omitempty"` +} + +func (x *RespondActivityTaskFailedRequest) Reset() { + *x = RespondActivityTaskFailedRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondActivityTaskFailedRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondActivityTaskFailedRequest) ProtoMessage() {} + +func (x *RespondActivityTaskFailedRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[32] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondActivityTaskFailedRequest.ProtoReflect.Descriptor instead. +func (*RespondActivityTaskFailedRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{32} +} + +func (x *RespondActivityTaskFailedRequest) GetTaskToken() []byte { + if x != nil { + return x.TaskToken + } + return nil +} + +func (x *RespondActivityTaskFailedRequest) GetFailure() *v15.Failure { + if x != nil { + return x.Failure + } + return nil +} + +func (x *RespondActivityTaskFailedRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *RespondActivityTaskFailedRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RespondActivityTaskFailedRequest) GetLastHeartbeatDetails() *v13.Payloads { + if x != nil { + return x.LastHeartbeatDetails + } + return nil +} + +func (x *RespondActivityTaskFailedRequest) GetWorkerVersion() *v13.WorkerVersionStamp { + if x != nil { + return x.WorkerVersion + } + return nil +} + +type RespondActivityTaskFailedResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Server validation failures could include + // last_heartbeat_details payload is too large, request failure is too large + Failures []*v15.Failure `protobuf:"bytes,1,rep,name=failures,proto3" json:"failures,omitempty"` +} + +func (x *RespondActivityTaskFailedResponse) Reset() { + *x = RespondActivityTaskFailedResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondActivityTaskFailedResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondActivityTaskFailedResponse) ProtoMessage() {} + +func (x *RespondActivityTaskFailedResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[33] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondActivityTaskFailedResponse.ProtoReflect.Descriptor instead. +func (*RespondActivityTaskFailedResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{33} +} + +func (x *RespondActivityTaskFailedResponse) GetFailures() []*v15.Failure { + if x != nil { + return x.Failures + } + return nil +} + +type RespondActivityTaskFailedByIdRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace of the workflow which scheduled this activity + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Id of the workflow which scheduled this activity + WorkflowId string `protobuf:"bytes,2,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + // Run Id of the workflow which scheduled this activity + RunId string `protobuf:"bytes,3,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` + // Id of the activity to fail + ActivityId string `protobuf:"bytes,4,opt,name=activity_id,json=activityId,proto3" json:"activity_id,omitempty"` + // Detailed failure information + Failure *v15.Failure `protobuf:"bytes,5,opt,name=failure,proto3" json:"failure,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,6,opt,name=identity,proto3" json:"identity,omitempty"` + // Additional details to be stored as last activity heartbeat + LastHeartbeatDetails *v13.Payloads `protobuf:"bytes,7,opt,name=last_heartbeat_details,json=lastHeartbeatDetails,proto3" json:"last_heartbeat_details,omitempty"` +} + +func (x *RespondActivityTaskFailedByIdRequest) Reset() { + *x = RespondActivityTaskFailedByIdRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondActivityTaskFailedByIdRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondActivityTaskFailedByIdRequest) ProtoMessage() {} + +func (x *RespondActivityTaskFailedByIdRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[34] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondActivityTaskFailedByIdRequest.ProtoReflect.Descriptor instead. +func (*RespondActivityTaskFailedByIdRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{34} +} + +func (x *RespondActivityTaskFailedByIdRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RespondActivityTaskFailedByIdRequest) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *RespondActivityTaskFailedByIdRequest) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +func (x *RespondActivityTaskFailedByIdRequest) GetActivityId() string { + if x != nil { + return x.ActivityId + } + return "" +} + +func (x *RespondActivityTaskFailedByIdRequest) GetFailure() *v15.Failure { + if x != nil { + return x.Failure + } + return nil +} + +func (x *RespondActivityTaskFailedByIdRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *RespondActivityTaskFailedByIdRequest) GetLastHeartbeatDetails() *v13.Payloads { + if x != nil { + return x.LastHeartbeatDetails + } + return nil +} + +type RespondActivityTaskFailedByIdResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Server validation failures could include + // last_heartbeat_details payload is too large, request failure is too large + Failures []*v15.Failure `protobuf:"bytes,1,rep,name=failures,proto3" json:"failures,omitempty"` +} + +func (x *RespondActivityTaskFailedByIdResponse) Reset() { + *x = RespondActivityTaskFailedByIdResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondActivityTaskFailedByIdResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondActivityTaskFailedByIdResponse) ProtoMessage() {} + +func (x *RespondActivityTaskFailedByIdResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[35] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondActivityTaskFailedByIdResponse.ProtoReflect.Descriptor instead. +func (*RespondActivityTaskFailedByIdResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{35} +} + +func (x *RespondActivityTaskFailedByIdResponse) GetFailures() []*v15.Failure { + if x != nil { + return x.Failures + } + return nil +} + +type RespondActivityTaskCanceledRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The task token as received in `PollActivityTaskQueueResponse` + TaskToken []byte `protobuf:"bytes,1,opt,name=task_token,json=taskToken,proto3" json:"task_token,omitempty"` + // Serialized additional information to attach to the cancellation + Details *v13.Payloads `protobuf:"bytes,2,opt,name=details,proto3" json:"details,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` + Namespace string `protobuf:"bytes,4,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Version info of the worker who processed this task. This message's `build_id` field should + // always be set by SDKs. Workers opting into versioning will also set the `use_versioning` + // field to true. See message docstrings for more. + WorkerVersion *v13.WorkerVersionStamp `protobuf:"bytes,5,opt,name=worker_version,json=workerVersion,proto3" json:"worker_version,omitempty"` +} + +func (x *RespondActivityTaskCanceledRequest) Reset() { + *x = RespondActivityTaskCanceledRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondActivityTaskCanceledRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondActivityTaskCanceledRequest) ProtoMessage() {} + +func (x *RespondActivityTaskCanceledRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[36] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondActivityTaskCanceledRequest.ProtoReflect.Descriptor instead. +func (*RespondActivityTaskCanceledRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{36} +} + +func (x *RespondActivityTaskCanceledRequest) GetTaskToken() []byte { + if x != nil { + return x.TaskToken + } + return nil +} + +func (x *RespondActivityTaskCanceledRequest) GetDetails() *v13.Payloads { + if x != nil { + return x.Details + } + return nil +} + +func (x *RespondActivityTaskCanceledRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *RespondActivityTaskCanceledRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RespondActivityTaskCanceledRequest) GetWorkerVersion() *v13.WorkerVersionStamp { + if x != nil { + return x.WorkerVersion + } + return nil +} + +type RespondActivityTaskCanceledResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RespondActivityTaskCanceledResponse) Reset() { + *x = RespondActivityTaskCanceledResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondActivityTaskCanceledResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondActivityTaskCanceledResponse) ProtoMessage() {} + +func (x *RespondActivityTaskCanceledResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[37] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondActivityTaskCanceledResponse.ProtoReflect.Descriptor instead. +func (*RespondActivityTaskCanceledResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{37} +} + +type RespondActivityTaskCanceledByIdRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace of the workflow which scheduled this activity + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Id of the workflow which scheduled this activity + WorkflowId string `protobuf:"bytes,2,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + // Run Id of the workflow which scheduled this activity + RunId string `protobuf:"bytes,3,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` + // Id of the activity to confirm is cancelled + ActivityId string `protobuf:"bytes,4,opt,name=activity_id,json=activityId,proto3" json:"activity_id,omitempty"` + // Serialized additional information to attach to the cancellation + Details *v13.Payloads `protobuf:"bytes,5,opt,name=details,proto3" json:"details,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,6,opt,name=identity,proto3" json:"identity,omitempty"` +} + +func (x *RespondActivityTaskCanceledByIdRequest) Reset() { + *x = RespondActivityTaskCanceledByIdRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondActivityTaskCanceledByIdRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondActivityTaskCanceledByIdRequest) ProtoMessage() {} + +func (x *RespondActivityTaskCanceledByIdRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[38] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondActivityTaskCanceledByIdRequest.ProtoReflect.Descriptor instead. +func (*RespondActivityTaskCanceledByIdRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{38} +} + +func (x *RespondActivityTaskCanceledByIdRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RespondActivityTaskCanceledByIdRequest) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *RespondActivityTaskCanceledByIdRequest) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +func (x *RespondActivityTaskCanceledByIdRequest) GetActivityId() string { + if x != nil { + return x.ActivityId + } + return "" +} + +func (x *RespondActivityTaskCanceledByIdRequest) GetDetails() *v13.Payloads { + if x != nil { + return x.Details + } + return nil +} + +func (x *RespondActivityTaskCanceledByIdRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +type RespondActivityTaskCanceledByIdResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RespondActivityTaskCanceledByIdResponse) Reset() { + *x = RespondActivityTaskCanceledByIdResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondActivityTaskCanceledByIdResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondActivityTaskCanceledByIdResponse) ProtoMessage() {} + +func (x *RespondActivityTaskCanceledByIdResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[39] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondActivityTaskCanceledByIdResponse.ProtoReflect.Descriptor instead. +func (*RespondActivityTaskCanceledByIdResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{39} +} + +type RequestCancelWorkflowExecutionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + WorkflowExecution *v13.WorkflowExecution `protobuf:"bytes,2,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` + // Used to de-dupe cancellation requests + RequestId string `protobuf:"bytes,4,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` + // If set, this call will error if the most recent (if no run id is set on + // `workflow_execution`), or specified (if it is) workflow execution is not part of the same + // execution chain as this id. + FirstExecutionRunId string `protobuf:"bytes,5,opt,name=first_execution_run_id,json=firstExecutionRunId,proto3" json:"first_execution_run_id,omitempty"` + // Reason for requesting the cancellation + Reason string `protobuf:"bytes,6,opt,name=reason,proto3" json:"reason,omitempty"` +} + +func (x *RequestCancelWorkflowExecutionRequest) Reset() { + *x = RequestCancelWorkflowExecutionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RequestCancelWorkflowExecutionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RequestCancelWorkflowExecutionRequest) ProtoMessage() {} + +func (x *RequestCancelWorkflowExecutionRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[40] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RequestCancelWorkflowExecutionRequest.ProtoReflect.Descriptor instead. +func (*RequestCancelWorkflowExecutionRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{40} +} + +func (x *RequestCancelWorkflowExecutionRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RequestCancelWorkflowExecutionRequest) GetWorkflowExecution() *v13.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *RequestCancelWorkflowExecutionRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *RequestCancelWorkflowExecutionRequest) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +func (x *RequestCancelWorkflowExecutionRequest) GetFirstExecutionRunId() string { + if x != nil { + return x.FirstExecutionRunId + } + return "" +} + +func (x *RequestCancelWorkflowExecutionRequest) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +type RequestCancelWorkflowExecutionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RequestCancelWorkflowExecutionResponse) Reset() { + *x = RequestCancelWorkflowExecutionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RequestCancelWorkflowExecutionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RequestCancelWorkflowExecutionResponse) ProtoMessage() {} + +func (x *RequestCancelWorkflowExecutionResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[41] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RequestCancelWorkflowExecutionResponse.ProtoReflect.Descriptor instead. +func (*RequestCancelWorkflowExecutionResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{41} +} + +type SignalWorkflowExecutionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + WorkflowExecution *v13.WorkflowExecution `protobuf:"bytes,2,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + // The workflow author-defined name of the signal to send to the workflow + SignalName string `protobuf:"bytes,3,opt,name=signal_name,json=signalName,proto3" json:"signal_name,omitempty"` + // Serialized value(s) to provide with the signal + Input *v13.Payloads `protobuf:"bytes,4,opt,name=input,proto3" json:"input,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,5,opt,name=identity,proto3" json:"identity,omitempty"` + // Used to de-dupe sent signals + RequestId string `protobuf:"bytes,6,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` + // Deprecated + Control string `protobuf:"bytes,7,opt,name=control,proto3" json:"control,omitempty"` + // Headers that are passed with the signal to the processing workflow. + // These can include things like auth or tracing tokens. + Header *v13.Header `protobuf:"bytes,8,opt,name=header,proto3" json:"header,omitempty"` + // Indicates that a new workflow task should not be generated when this signal is received. + SkipGenerateWorkflowTask bool `protobuf:"varint,9,opt,name=skip_generate_workflow_task,json=skipGenerateWorkflowTask,proto3" json:"skip_generate_workflow_task,omitempty"` +} + +func (x *SignalWorkflowExecutionRequest) Reset() { + *x = SignalWorkflowExecutionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SignalWorkflowExecutionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SignalWorkflowExecutionRequest) ProtoMessage() {} + +func (x *SignalWorkflowExecutionRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[42] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SignalWorkflowExecutionRequest.ProtoReflect.Descriptor instead. +func (*SignalWorkflowExecutionRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{42} +} + +func (x *SignalWorkflowExecutionRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *SignalWorkflowExecutionRequest) GetWorkflowExecution() *v13.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *SignalWorkflowExecutionRequest) GetSignalName() string { + if x != nil { + return x.SignalName + } + return "" +} + +func (x *SignalWorkflowExecutionRequest) GetInput() *v13.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *SignalWorkflowExecutionRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *SignalWorkflowExecutionRequest) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +func (x *SignalWorkflowExecutionRequest) GetControl() string { + if x != nil { + return x.Control + } + return "" +} + +func (x *SignalWorkflowExecutionRequest) GetHeader() *v13.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *SignalWorkflowExecutionRequest) GetSkipGenerateWorkflowTask() bool { + if x != nil { + return x.SkipGenerateWorkflowTask + } + return false +} + +type SignalWorkflowExecutionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *SignalWorkflowExecutionResponse) Reset() { + *x = SignalWorkflowExecutionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SignalWorkflowExecutionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SignalWorkflowExecutionResponse) ProtoMessage() {} + +func (x *SignalWorkflowExecutionResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[43] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SignalWorkflowExecutionResponse.ProtoReflect.Descriptor instead. +func (*SignalWorkflowExecutionResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{43} +} + +type SignalWithStartWorkflowExecutionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + WorkflowId string `protobuf:"bytes,2,opt,name=workflow_id,json=workflowId,proto3" json:"workflow_id,omitempty"` + WorkflowType *v13.WorkflowType `protobuf:"bytes,3,opt,name=workflow_type,json=workflowType,proto3" json:"workflow_type,omitempty"` + // The task queue to start this workflow on, if it will be started + TaskQueue *v14.TaskQueue `protobuf:"bytes,4,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + // Serialized arguments to the workflow. These are passed as arguments to the workflow function. + Input *v13.Payloads `protobuf:"bytes,5,opt,name=input,proto3" json:"input,omitempty"` + // Total workflow execution timeout including retries and continue as new + WorkflowExecutionTimeout *durationpb.Duration `protobuf:"bytes,6,opt,name=workflow_execution_timeout,json=workflowExecutionTimeout,proto3" json:"workflow_execution_timeout,omitempty"` + // Timeout of a single workflow run + WorkflowRunTimeout *durationpb.Duration `protobuf:"bytes,7,opt,name=workflow_run_timeout,json=workflowRunTimeout,proto3" json:"workflow_run_timeout,omitempty"` + // Timeout of a single workflow task + WorkflowTaskTimeout *durationpb.Duration `protobuf:"bytes,8,opt,name=workflow_task_timeout,json=workflowTaskTimeout,proto3" json:"workflow_task_timeout,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,9,opt,name=identity,proto3" json:"identity,omitempty"` + // Used to de-dupe signal w/ start requests + RequestId string `protobuf:"bytes,10,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` + // Defines whether to allow re-using the workflow id from a previously *closed* workflow. + // The default policy is WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE. + // + // See `workflow_id_reuse_policy` for handling a workflow id duplication with a *running* workflow. + WorkflowIdReusePolicy v11.WorkflowIdReusePolicy `protobuf:"varint,11,opt,name=workflow_id_reuse_policy,json=workflowIdReusePolicy,proto3,enum=temporal.api.enums.v1.WorkflowIdReusePolicy" json:"workflow_id_reuse_policy,omitempty"` + // Defines how to resolve a workflow id conflict with a *running* workflow. + // The default policy is WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING. + // Note that WORKFLOW_ID_CONFLICT_POLICY_FAIL is an invalid option. + // + // See `workflow_id_reuse_policy` for handling a workflow id duplication with a *closed* workflow. + WorkflowIdConflictPolicy v11.WorkflowIdConflictPolicy `protobuf:"varint,22,opt,name=workflow_id_conflict_policy,json=workflowIdConflictPolicy,proto3,enum=temporal.api.enums.v1.WorkflowIdConflictPolicy" json:"workflow_id_conflict_policy,omitempty"` + // The workflow author-defined name of the signal to send to the workflow + SignalName string `protobuf:"bytes,12,opt,name=signal_name,json=signalName,proto3" json:"signal_name,omitempty"` + // Serialized value(s) to provide with the signal + SignalInput *v13.Payloads `protobuf:"bytes,13,opt,name=signal_input,json=signalInput,proto3" json:"signal_input,omitempty"` + // Deprecated + Control string `protobuf:"bytes,14,opt,name=control,proto3" json:"control,omitempty"` + // Retry policy for the workflow + RetryPolicy *v13.RetryPolicy `protobuf:"bytes,15,opt,name=retry_policy,json=retryPolicy,proto3" json:"retry_policy,omitempty"` + // See https://docs.temporal.io/docs/content/what-is-a-temporal-cron-job/ + CronSchedule string `protobuf:"bytes,16,opt,name=cron_schedule,json=cronSchedule,proto3" json:"cron_schedule,omitempty"` + Memo *v13.Memo `protobuf:"bytes,17,opt,name=memo,proto3" json:"memo,omitempty"` + SearchAttributes *v13.SearchAttributes `protobuf:"bytes,18,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` + Header *v13.Header `protobuf:"bytes,19,opt,name=header,proto3" json:"header,omitempty"` + // Time to wait before dispatching the first workflow task. Cannot be used with `cron_schedule`. + // Note that the signal will be delivered with the first workflow task. If the workflow gets + // another SignalWithStartWorkflow before the delay and `skip_generate_workflow_task` is false + // or not set, a workflow task will be dispatched immediately and the rest of the delay period + // will be ignored, even if that request also had a delay. Signal via SignalWorkflowExecution + // will not unblock the workflow. + WorkflowStartDelay *durationpb.Duration `protobuf:"bytes,20,opt,name=workflow_start_delay,json=workflowStartDelay,proto3" json:"workflow_start_delay,omitempty"` + // Indicates that a new workflow task should not be generated when this signal is received. + SkipGenerateWorkflowTask bool `protobuf:"varint,21,opt,name=skip_generate_workflow_task,json=skipGenerateWorkflowTask,proto3" json:"skip_generate_workflow_task,omitempty"` + // Metadata on the workflow if it is started. This is carried over to the WorkflowExecutionInfo + // for use by user interfaces to display the fixed as-of-start summary and details of the + // workflow. + UserMetadata *v16.UserMetadata `protobuf:"bytes,23,opt,name=user_metadata,json=userMetadata,proto3" json:"user_metadata,omitempty"` +} + +func (x *SignalWithStartWorkflowExecutionRequest) Reset() { + *x = SignalWithStartWorkflowExecutionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SignalWithStartWorkflowExecutionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SignalWithStartWorkflowExecutionRequest) ProtoMessage() {} + +func (x *SignalWithStartWorkflowExecutionRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[44] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SignalWithStartWorkflowExecutionRequest.ProtoReflect.Descriptor instead. +func (*SignalWithStartWorkflowExecutionRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{44} +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetWorkflowType() *v13.WorkflowType { + if x != nil { + return x.WorkflowType + } + return nil +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetTaskQueue() *v14.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetInput() *v13.Payloads { + if x != nil { + return x.Input + } + return nil +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetWorkflowExecutionTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowExecutionTimeout + } + return nil +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetWorkflowRunTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowRunTimeout + } + return nil +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetWorkflowTaskTimeout() *durationpb.Duration { + if x != nil { + return x.WorkflowTaskTimeout + } + return nil +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetWorkflowIdReusePolicy() v11.WorkflowIdReusePolicy { + if x != nil { + return x.WorkflowIdReusePolicy + } + return v11.WorkflowIdReusePolicy(0) +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetWorkflowIdConflictPolicy() v11.WorkflowIdConflictPolicy { + if x != nil { + return x.WorkflowIdConflictPolicy + } + return v11.WorkflowIdConflictPolicy(0) +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetSignalName() string { + if x != nil { + return x.SignalName + } + return "" +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetSignalInput() *v13.Payloads { + if x != nil { + return x.SignalInput + } + return nil +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetControl() string { + if x != nil { + return x.Control + } + return "" +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetRetryPolicy() *v13.RetryPolicy { + if x != nil { + return x.RetryPolicy + } + return nil +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetCronSchedule() string { + if x != nil { + return x.CronSchedule + } + return "" +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetMemo() *v13.Memo { + if x != nil { + return x.Memo + } + return nil +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetSearchAttributes() *v13.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetHeader() *v13.Header { + if x != nil { + return x.Header + } + return nil +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetWorkflowStartDelay() *durationpb.Duration { + if x != nil { + return x.WorkflowStartDelay + } + return nil +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetSkipGenerateWorkflowTask() bool { + if x != nil { + return x.SkipGenerateWorkflowTask + } + return false +} + +func (x *SignalWithStartWorkflowExecutionRequest) GetUserMetadata() *v16.UserMetadata { + if x != nil { + return x.UserMetadata + } + return nil +} + +type SignalWithStartWorkflowExecutionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The run id of the workflow that was started - or just signaled, if it was already running. + RunId string `protobuf:"bytes,1,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` + // If true, a new workflow was started. + Started bool `protobuf:"varint,2,opt,name=started,proto3" json:"started,omitempty"` +} + +func (x *SignalWithStartWorkflowExecutionResponse) Reset() { + *x = SignalWithStartWorkflowExecutionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SignalWithStartWorkflowExecutionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SignalWithStartWorkflowExecutionResponse) ProtoMessage() {} + +func (x *SignalWithStartWorkflowExecutionResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[45] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SignalWithStartWorkflowExecutionResponse.ProtoReflect.Descriptor instead. +func (*SignalWithStartWorkflowExecutionResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{45} +} + +func (x *SignalWithStartWorkflowExecutionResponse) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +func (x *SignalWithStartWorkflowExecutionResponse) GetStarted() bool { + if x != nil { + return x.Started + } + return false +} + +type ResetWorkflowExecutionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + WorkflowExecution *v13.WorkflowExecution `protobuf:"bytes,2,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + Reason string `protobuf:"bytes,3,opt,name=reason,proto3" json:"reason,omitempty"` + // The id of a `WORKFLOW_TASK_COMPLETED`,`WORKFLOW_TASK_TIMED_OUT`, `WORKFLOW_TASK_FAILED`, or + // `WORKFLOW_TASK_STARTED` event to reset to. + WorkflowTaskFinishEventId int64 `protobuf:"varint,4,opt,name=workflow_task_finish_event_id,json=workflowTaskFinishEventId,proto3" json:"workflow_task_finish_event_id,omitempty"` + // Used to de-dupe reset requests + RequestId string `protobuf:"bytes,5,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` + // Event types to be reapplied (deprecated) + // Default: RESET_REAPPLY_TYPE_SIGNAL + ResetReapplyType v11.ResetReapplyType `protobuf:"varint,6,opt,name=reset_reapply_type,json=resetReapplyType,proto3,enum=temporal.api.enums.v1.ResetReapplyType" json:"reset_reapply_type,omitempty"` + // Event types not to be reapplied + ResetReapplyExcludeTypes []v11.ResetReapplyExcludeType `protobuf:"varint,7,rep,packed,name=reset_reapply_exclude_types,json=resetReapplyExcludeTypes,proto3,enum=temporal.api.enums.v1.ResetReapplyExcludeType" json:"reset_reapply_exclude_types,omitempty"` +} + +func (x *ResetWorkflowExecutionRequest) Reset() { + *x = ResetWorkflowExecutionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[46] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResetWorkflowExecutionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResetWorkflowExecutionRequest) ProtoMessage() {} + +func (x *ResetWorkflowExecutionRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[46] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResetWorkflowExecutionRequest.ProtoReflect.Descriptor instead. +func (*ResetWorkflowExecutionRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{46} +} + +func (x *ResetWorkflowExecutionRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ResetWorkflowExecutionRequest) GetWorkflowExecution() *v13.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *ResetWorkflowExecutionRequest) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *ResetWorkflowExecutionRequest) GetWorkflowTaskFinishEventId() int64 { + if x != nil { + return x.WorkflowTaskFinishEventId + } + return 0 +} + +func (x *ResetWorkflowExecutionRequest) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +func (x *ResetWorkflowExecutionRequest) GetResetReapplyType() v11.ResetReapplyType { + if x != nil { + return x.ResetReapplyType + } + return v11.ResetReapplyType(0) +} + +func (x *ResetWorkflowExecutionRequest) GetResetReapplyExcludeTypes() []v11.ResetReapplyExcludeType { + if x != nil { + return x.ResetReapplyExcludeTypes + } + return nil +} + +type ResetWorkflowExecutionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + RunId string `protobuf:"bytes,1,opt,name=run_id,json=runId,proto3" json:"run_id,omitempty"` +} + +func (x *ResetWorkflowExecutionResponse) Reset() { + *x = ResetWorkflowExecutionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[47] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResetWorkflowExecutionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResetWorkflowExecutionResponse) ProtoMessage() {} + +func (x *ResetWorkflowExecutionResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[47] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResetWorkflowExecutionResponse.ProtoReflect.Descriptor instead. +func (*ResetWorkflowExecutionResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{47} +} + +func (x *ResetWorkflowExecutionResponse) GetRunId() string { + if x != nil { + return x.RunId + } + return "" +} + +type TerminateWorkflowExecutionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + WorkflowExecution *v13.WorkflowExecution `protobuf:"bytes,2,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + Reason string `protobuf:"bytes,3,opt,name=reason,proto3" json:"reason,omitempty"` + // Serialized additional information to attach to the termination event + Details *v13.Payloads `protobuf:"bytes,4,opt,name=details,proto3" json:"details,omitempty"` + // The identity of the worker/client + Identity string `protobuf:"bytes,5,opt,name=identity,proto3" json:"identity,omitempty"` + // If set, this call will error if the most recent (if no run id is set on + // `workflow_execution`), or specified (if it is) workflow execution is not part of the same + // execution chain as this id. + FirstExecutionRunId string `protobuf:"bytes,6,opt,name=first_execution_run_id,json=firstExecutionRunId,proto3" json:"first_execution_run_id,omitempty"` +} + +func (x *TerminateWorkflowExecutionRequest) Reset() { + *x = TerminateWorkflowExecutionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[48] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TerminateWorkflowExecutionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TerminateWorkflowExecutionRequest) ProtoMessage() {} + +func (x *TerminateWorkflowExecutionRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[48] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TerminateWorkflowExecutionRequest.ProtoReflect.Descriptor instead. +func (*TerminateWorkflowExecutionRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{48} +} + +func (x *TerminateWorkflowExecutionRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *TerminateWorkflowExecutionRequest) GetWorkflowExecution() *v13.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *TerminateWorkflowExecutionRequest) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *TerminateWorkflowExecutionRequest) GetDetails() *v13.Payloads { + if x != nil { + return x.Details + } + return nil +} + +func (x *TerminateWorkflowExecutionRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *TerminateWorkflowExecutionRequest) GetFirstExecutionRunId() string { + if x != nil { + return x.FirstExecutionRunId + } + return "" +} + +type TerminateWorkflowExecutionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *TerminateWorkflowExecutionResponse) Reset() { + *x = TerminateWorkflowExecutionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[49] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TerminateWorkflowExecutionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TerminateWorkflowExecutionResponse) ProtoMessage() {} + +func (x *TerminateWorkflowExecutionResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[49] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TerminateWorkflowExecutionResponse.ProtoReflect.Descriptor instead. +func (*TerminateWorkflowExecutionResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{49} +} + +type DeleteWorkflowExecutionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Workflow Execution to delete. If run_id is not specified, the latest one is used. + WorkflowExecution *v13.WorkflowExecution `protobuf:"bytes,2,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` +} + +func (x *DeleteWorkflowExecutionRequest) Reset() { + *x = DeleteWorkflowExecutionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[50] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteWorkflowExecutionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteWorkflowExecutionRequest) ProtoMessage() {} + +func (x *DeleteWorkflowExecutionRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[50] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteWorkflowExecutionRequest.ProtoReflect.Descriptor instead. +func (*DeleteWorkflowExecutionRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{50} +} + +func (x *DeleteWorkflowExecutionRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *DeleteWorkflowExecutionRequest) GetWorkflowExecution() *v13.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +type DeleteWorkflowExecutionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *DeleteWorkflowExecutionResponse) Reset() { + *x = DeleteWorkflowExecutionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[51] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteWorkflowExecutionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteWorkflowExecutionResponse) ProtoMessage() {} + +func (x *DeleteWorkflowExecutionResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[51] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteWorkflowExecutionResponse.ProtoReflect.Descriptor instead. +func (*DeleteWorkflowExecutionResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{51} +} + +type ListOpenWorkflowExecutionsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + MaximumPageSize int32 `protobuf:"varint,2,opt,name=maximum_page_size,json=maximumPageSize,proto3" json:"maximum_page_size,omitempty"` + NextPageToken []byte `protobuf:"bytes,3,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` + StartTimeFilter *v111.StartTimeFilter `protobuf:"bytes,4,opt,name=start_time_filter,json=startTimeFilter,proto3" json:"start_time_filter,omitempty"` + // Types that are assignable to Filters: + // + // *ListOpenWorkflowExecutionsRequest_ExecutionFilter + // *ListOpenWorkflowExecutionsRequest_TypeFilter + Filters isListOpenWorkflowExecutionsRequest_Filters `protobuf_oneof:"filters"` +} + +func (x *ListOpenWorkflowExecutionsRequest) Reset() { + *x = ListOpenWorkflowExecutionsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[52] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListOpenWorkflowExecutionsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOpenWorkflowExecutionsRequest) ProtoMessage() {} + +func (x *ListOpenWorkflowExecutionsRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[52] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOpenWorkflowExecutionsRequest.ProtoReflect.Descriptor instead. +func (*ListOpenWorkflowExecutionsRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{52} +} + +func (x *ListOpenWorkflowExecutionsRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ListOpenWorkflowExecutionsRequest) GetMaximumPageSize() int32 { + if x != nil { + return x.MaximumPageSize + } + return 0 +} + +func (x *ListOpenWorkflowExecutionsRequest) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +func (x *ListOpenWorkflowExecutionsRequest) GetStartTimeFilter() *v111.StartTimeFilter { + if x != nil { + return x.StartTimeFilter + } + return nil +} + +func (m *ListOpenWorkflowExecutionsRequest) GetFilters() isListOpenWorkflowExecutionsRequest_Filters { + if m != nil { + return m.Filters + } + return nil +} + +func (x *ListOpenWorkflowExecutionsRequest) GetExecutionFilter() *v111.WorkflowExecutionFilter { + if x, ok := x.GetFilters().(*ListOpenWorkflowExecutionsRequest_ExecutionFilter); ok { + return x.ExecutionFilter + } + return nil +} + +func (x *ListOpenWorkflowExecutionsRequest) GetTypeFilter() *v111.WorkflowTypeFilter { + if x, ok := x.GetFilters().(*ListOpenWorkflowExecutionsRequest_TypeFilter); ok { + return x.TypeFilter + } + return nil +} + +type isListOpenWorkflowExecutionsRequest_Filters interface { + isListOpenWorkflowExecutionsRequest_Filters() +} + +type ListOpenWorkflowExecutionsRequest_ExecutionFilter struct { + ExecutionFilter *v111.WorkflowExecutionFilter `protobuf:"bytes,5,opt,name=execution_filter,json=executionFilter,proto3,oneof"` +} + +type ListOpenWorkflowExecutionsRequest_TypeFilter struct { + TypeFilter *v111.WorkflowTypeFilter `protobuf:"bytes,6,opt,name=type_filter,json=typeFilter,proto3,oneof"` +} + +func (*ListOpenWorkflowExecutionsRequest_ExecutionFilter) isListOpenWorkflowExecutionsRequest_Filters() { +} + +func (*ListOpenWorkflowExecutionsRequest_TypeFilter) isListOpenWorkflowExecutionsRequest_Filters() {} + +type ListOpenWorkflowExecutionsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Executions []*v112.WorkflowExecutionInfo `protobuf:"bytes,1,rep,name=executions,proto3" json:"executions,omitempty"` + NextPageToken []byte `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *ListOpenWorkflowExecutionsResponse) Reset() { + *x = ListOpenWorkflowExecutionsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[53] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListOpenWorkflowExecutionsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListOpenWorkflowExecutionsResponse) ProtoMessage() {} + +func (x *ListOpenWorkflowExecutionsResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[53] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListOpenWorkflowExecutionsResponse.ProtoReflect.Descriptor instead. +func (*ListOpenWorkflowExecutionsResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{53} +} + +func (x *ListOpenWorkflowExecutionsResponse) GetExecutions() []*v112.WorkflowExecutionInfo { + if x != nil { + return x.Executions + } + return nil +} + +func (x *ListOpenWorkflowExecutionsResponse) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +type ListClosedWorkflowExecutionsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + MaximumPageSize int32 `protobuf:"varint,2,opt,name=maximum_page_size,json=maximumPageSize,proto3" json:"maximum_page_size,omitempty"` + NextPageToken []byte `protobuf:"bytes,3,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` + StartTimeFilter *v111.StartTimeFilter `protobuf:"bytes,4,opt,name=start_time_filter,json=startTimeFilter,proto3" json:"start_time_filter,omitempty"` + // Types that are assignable to Filters: + // + // *ListClosedWorkflowExecutionsRequest_ExecutionFilter + // *ListClosedWorkflowExecutionsRequest_TypeFilter + // *ListClosedWorkflowExecutionsRequest_StatusFilter + Filters isListClosedWorkflowExecutionsRequest_Filters `protobuf_oneof:"filters"` +} + +func (x *ListClosedWorkflowExecutionsRequest) Reset() { + *x = ListClosedWorkflowExecutionsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[54] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListClosedWorkflowExecutionsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListClosedWorkflowExecutionsRequest) ProtoMessage() {} + +func (x *ListClosedWorkflowExecutionsRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[54] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListClosedWorkflowExecutionsRequest.ProtoReflect.Descriptor instead. +func (*ListClosedWorkflowExecutionsRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{54} +} + +func (x *ListClosedWorkflowExecutionsRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ListClosedWorkflowExecutionsRequest) GetMaximumPageSize() int32 { + if x != nil { + return x.MaximumPageSize + } + return 0 +} + +func (x *ListClosedWorkflowExecutionsRequest) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +func (x *ListClosedWorkflowExecutionsRequest) GetStartTimeFilter() *v111.StartTimeFilter { + if x != nil { + return x.StartTimeFilter + } + return nil +} + +func (m *ListClosedWorkflowExecutionsRequest) GetFilters() isListClosedWorkflowExecutionsRequest_Filters { + if m != nil { + return m.Filters + } + return nil +} + +func (x *ListClosedWorkflowExecutionsRequest) GetExecutionFilter() *v111.WorkflowExecutionFilter { + if x, ok := x.GetFilters().(*ListClosedWorkflowExecutionsRequest_ExecutionFilter); ok { + return x.ExecutionFilter + } + return nil +} + +func (x *ListClosedWorkflowExecutionsRequest) GetTypeFilter() *v111.WorkflowTypeFilter { + if x, ok := x.GetFilters().(*ListClosedWorkflowExecutionsRequest_TypeFilter); ok { + return x.TypeFilter + } + return nil +} + +func (x *ListClosedWorkflowExecutionsRequest) GetStatusFilter() *v111.StatusFilter { + if x, ok := x.GetFilters().(*ListClosedWorkflowExecutionsRequest_StatusFilter); ok { + return x.StatusFilter + } + return nil +} + +type isListClosedWorkflowExecutionsRequest_Filters interface { + isListClosedWorkflowExecutionsRequest_Filters() +} + +type ListClosedWorkflowExecutionsRequest_ExecutionFilter struct { + ExecutionFilter *v111.WorkflowExecutionFilter `protobuf:"bytes,5,opt,name=execution_filter,json=executionFilter,proto3,oneof"` +} + +type ListClosedWorkflowExecutionsRequest_TypeFilter struct { + TypeFilter *v111.WorkflowTypeFilter `protobuf:"bytes,6,opt,name=type_filter,json=typeFilter,proto3,oneof"` +} + +type ListClosedWorkflowExecutionsRequest_StatusFilter struct { + StatusFilter *v111.StatusFilter `protobuf:"bytes,7,opt,name=status_filter,json=statusFilter,proto3,oneof"` +} + +func (*ListClosedWorkflowExecutionsRequest_ExecutionFilter) isListClosedWorkflowExecutionsRequest_Filters() { +} + +func (*ListClosedWorkflowExecutionsRequest_TypeFilter) isListClosedWorkflowExecutionsRequest_Filters() { +} + +func (*ListClosedWorkflowExecutionsRequest_StatusFilter) isListClosedWorkflowExecutionsRequest_Filters() { +} + +type ListClosedWorkflowExecutionsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Executions []*v112.WorkflowExecutionInfo `protobuf:"bytes,1,rep,name=executions,proto3" json:"executions,omitempty"` + NextPageToken []byte `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *ListClosedWorkflowExecutionsResponse) Reset() { + *x = ListClosedWorkflowExecutionsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[55] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListClosedWorkflowExecutionsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListClosedWorkflowExecutionsResponse) ProtoMessage() {} + +func (x *ListClosedWorkflowExecutionsResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[55] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListClosedWorkflowExecutionsResponse.ProtoReflect.Descriptor instead. +func (*ListClosedWorkflowExecutionsResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{55} +} + +func (x *ListClosedWorkflowExecutionsResponse) GetExecutions() []*v112.WorkflowExecutionInfo { + if x != nil { + return x.Executions + } + return nil +} + +func (x *ListClosedWorkflowExecutionsResponse) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +type ListWorkflowExecutionsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + PageSize int32 `protobuf:"varint,2,opt,name=page_size,json=pageSize,proto3" json:"page_size,omitempty"` + NextPageToken []byte `protobuf:"bytes,3,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` + Query string `protobuf:"bytes,4,opt,name=query,proto3" json:"query,omitempty"` +} + +func (x *ListWorkflowExecutionsRequest) Reset() { + *x = ListWorkflowExecutionsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[56] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListWorkflowExecutionsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListWorkflowExecutionsRequest) ProtoMessage() {} + +func (x *ListWorkflowExecutionsRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[56] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListWorkflowExecutionsRequest.ProtoReflect.Descriptor instead. +func (*ListWorkflowExecutionsRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{56} +} + +func (x *ListWorkflowExecutionsRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ListWorkflowExecutionsRequest) GetPageSize() int32 { + if x != nil { + return x.PageSize + } + return 0 +} + +func (x *ListWorkflowExecutionsRequest) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +func (x *ListWorkflowExecutionsRequest) GetQuery() string { + if x != nil { + return x.Query + } + return "" +} + +type ListWorkflowExecutionsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Executions []*v112.WorkflowExecutionInfo `protobuf:"bytes,1,rep,name=executions,proto3" json:"executions,omitempty"` + NextPageToken []byte `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *ListWorkflowExecutionsResponse) Reset() { + *x = ListWorkflowExecutionsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[57] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListWorkflowExecutionsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListWorkflowExecutionsResponse) ProtoMessage() {} + +func (x *ListWorkflowExecutionsResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[57] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListWorkflowExecutionsResponse.ProtoReflect.Descriptor instead. +func (*ListWorkflowExecutionsResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{57} +} + +func (x *ListWorkflowExecutionsResponse) GetExecutions() []*v112.WorkflowExecutionInfo { + if x != nil { + return x.Executions + } + return nil +} + +func (x *ListWorkflowExecutionsResponse) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +type ListArchivedWorkflowExecutionsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + PageSize int32 `protobuf:"varint,2,opt,name=page_size,json=pageSize,proto3" json:"page_size,omitempty"` + NextPageToken []byte `protobuf:"bytes,3,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` + Query string `protobuf:"bytes,4,opt,name=query,proto3" json:"query,omitempty"` +} + +func (x *ListArchivedWorkflowExecutionsRequest) Reset() { + *x = ListArchivedWorkflowExecutionsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[58] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListArchivedWorkflowExecutionsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListArchivedWorkflowExecutionsRequest) ProtoMessage() {} + +func (x *ListArchivedWorkflowExecutionsRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[58] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListArchivedWorkflowExecutionsRequest.ProtoReflect.Descriptor instead. +func (*ListArchivedWorkflowExecutionsRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{58} +} + +func (x *ListArchivedWorkflowExecutionsRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ListArchivedWorkflowExecutionsRequest) GetPageSize() int32 { + if x != nil { + return x.PageSize + } + return 0 +} + +func (x *ListArchivedWorkflowExecutionsRequest) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +func (x *ListArchivedWorkflowExecutionsRequest) GetQuery() string { + if x != nil { + return x.Query + } + return "" +} + +type ListArchivedWorkflowExecutionsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Executions []*v112.WorkflowExecutionInfo `protobuf:"bytes,1,rep,name=executions,proto3" json:"executions,omitempty"` + NextPageToken []byte `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *ListArchivedWorkflowExecutionsResponse) Reset() { + *x = ListArchivedWorkflowExecutionsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[59] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListArchivedWorkflowExecutionsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListArchivedWorkflowExecutionsResponse) ProtoMessage() {} + +func (x *ListArchivedWorkflowExecutionsResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[59] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListArchivedWorkflowExecutionsResponse.ProtoReflect.Descriptor instead. +func (*ListArchivedWorkflowExecutionsResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{59} +} + +func (x *ListArchivedWorkflowExecutionsResponse) GetExecutions() []*v112.WorkflowExecutionInfo { + if x != nil { + return x.Executions + } + return nil +} + +func (x *ListArchivedWorkflowExecutionsResponse) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +type ScanWorkflowExecutionsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + PageSize int32 `protobuf:"varint,2,opt,name=page_size,json=pageSize,proto3" json:"page_size,omitempty"` + NextPageToken []byte `protobuf:"bytes,3,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` + Query string `protobuf:"bytes,4,opt,name=query,proto3" json:"query,omitempty"` +} + +func (x *ScanWorkflowExecutionsRequest) Reset() { + *x = ScanWorkflowExecutionsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[60] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ScanWorkflowExecutionsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ScanWorkflowExecutionsRequest) ProtoMessage() {} + +func (x *ScanWorkflowExecutionsRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[60] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ScanWorkflowExecutionsRequest.ProtoReflect.Descriptor instead. +func (*ScanWorkflowExecutionsRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{60} +} + +func (x *ScanWorkflowExecutionsRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ScanWorkflowExecutionsRequest) GetPageSize() int32 { + if x != nil { + return x.PageSize + } + return 0 +} + +func (x *ScanWorkflowExecutionsRequest) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +func (x *ScanWorkflowExecutionsRequest) GetQuery() string { + if x != nil { + return x.Query + } + return "" +} + +type ScanWorkflowExecutionsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Executions []*v112.WorkflowExecutionInfo `protobuf:"bytes,1,rep,name=executions,proto3" json:"executions,omitempty"` + NextPageToken []byte `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *ScanWorkflowExecutionsResponse) Reset() { + *x = ScanWorkflowExecutionsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[61] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ScanWorkflowExecutionsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ScanWorkflowExecutionsResponse) ProtoMessage() {} + +func (x *ScanWorkflowExecutionsResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[61] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ScanWorkflowExecutionsResponse.ProtoReflect.Descriptor instead. +func (*ScanWorkflowExecutionsResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{61} +} + +func (x *ScanWorkflowExecutionsResponse) GetExecutions() []*v112.WorkflowExecutionInfo { + if x != nil { + return x.Executions + } + return nil +} + +func (x *ScanWorkflowExecutionsResponse) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +type CountWorkflowExecutionsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + Query string `protobuf:"bytes,2,opt,name=query,proto3" json:"query,omitempty"` +} + +func (x *CountWorkflowExecutionsRequest) Reset() { + *x = CountWorkflowExecutionsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[62] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CountWorkflowExecutionsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CountWorkflowExecutionsRequest) ProtoMessage() {} + +func (x *CountWorkflowExecutionsRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[62] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CountWorkflowExecutionsRequest.ProtoReflect.Descriptor instead. +func (*CountWorkflowExecutionsRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{62} +} + +func (x *CountWorkflowExecutionsRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *CountWorkflowExecutionsRequest) GetQuery() string { + if x != nil { + return x.Query + } + return "" +} + +type CountWorkflowExecutionsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // If `query` is not grouping by any field, the count is an approximate number + // of workflows that matches the query. + // If `query` is grouping by a field, the count is simply the sum of the counts + // of the groups returned in the response. This number can be smaller than the + // total number of workflows matching the query. + Count int64 `protobuf:"varint,1,opt,name=count,proto3" json:"count,omitempty"` + // `groups` contains the groups if the request is grouping by a field. + // The list might not be complete, and the counts of each group is approximate. + Groups []*CountWorkflowExecutionsResponse_AggregationGroup `protobuf:"bytes,2,rep,name=groups,proto3" json:"groups,omitempty"` +} + +func (x *CountWorkflowExecutionsResponse) Reset() { + *x = CountWorkflowExecutionsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[63] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CountWorkflowExecutionsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CountWorkflowExecutionsResponse) ProtoMessage() {} + +func (x *CountWorkflowExecutionsResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[63] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CountWorkflowExecutionsResponse.ProtoReflect.Descriptor instead. +func (*CountWorkflowExecutionsResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{63} +} + +func (x *CountWorkflowExecutionsResponse) GetCount() int64 { + if x != nil { + return x.Count + } + return 0 +} + +func (x *CountWorkflowExecutionsResponse) GetGroups() []*CountWorkflowExecutionsResponse_AggregationGroup { + if x != nil { + return x.Groups + } + return nil +} + +type GetSearchAttributesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *GetSearchAttributesRequest) Reset() { + *x = GetSearchAttributesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[64] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSearchAttributesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSearchAttributesRequest) ProtoMessage() {} + +func (x *GetSearchAttributesRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[64] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetSearchAttributesRequest.ProtoReflect.Descriptor instead. +func (*GetSearchAttributesRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{64} +} + +type GetSearchAttributesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Keys map[string]v11.IndexedValueType `protobuf:"bytes,1,rep,name=keys,proto3" json:"keys,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3,enum=temporal.api.enums.v1.IndexedValueType"` +} + +func (x *GetSearchAttributesResponse) Reset() { + *x = GetSearchAttributesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[65] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSearchAttributesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSearchAttributesResponse) ProtoMessage() {} + +func (x *GetSearchAttributesResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[65] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetSearchAttributesResponse.ProtoReflect.Descriptor instead. +func (*GetSearchAttributesResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{65} +} + +func (x *GetSearchAttributesResponse) GetKeys() map[string]v11.IndexedValueType { + if x != nil { + return x.Keys + } + return nil +} + +type RespondQueryTaskCompletedRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TaskToken []byte `protobuf:"bytes,1,opt,name=task_token,json=taskToken,proto3" json:"task_token,omitempty"` + CompletedType v11.QueryResultType `protobuf:"varint,2,opt,name=completed_type,json=completedType,proto3,enum=temporal.api.enums.v1.QueryResultType" json:"completed_type,omitempty"` + QueryResult *v13.Payloads `protobuf:"bytes,3,opt,name=query_result,json=queryResult,proto3" json:"query_result,omitempty"` + ErrorMessage string `protobuf:"bytes,4,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"` + Namespace string `protobuf:"bytes,6,opt,name=namespace,proto3" json:"namespace,omitempty"` +} + +func (x *RespondQueryTaskCompletedRequest) Reset() { + *x = RespondQueryTaskCompletedRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[66] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondQueryTaskCompletedRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondQueryTaskCompletedRequest) ProtoMessage() {} + +func (x *RespondQueryTaskCompletedRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[66] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondQueryTaskCompletedRequest.ProtoReflect.Descriptor instead. +func (*RespondQueryTaskCompletedRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{66} +} + +func (x *RespondQueryTaskCompletedRequest) GetTaskToken() []byte { + if x != nil { + return x.TaskToken + } + return nil +} + +func (x *RespondQueryTaskCompletedRequest) GetCompletedType() v11.QueryResultType { + if x != nil { + return x.CompletedType + } + return v11.QueryResultType(0) +} + +func (x *RespondQueryTaskCompletedRequest) GetQueryResult() *v13.Payloads { + if x != nil { + return x.QueryResult + } + return nil +} + +func (x *RespondQueryTaskCompletedRequest) GetErrorMessage() string { + if x != nil { + return x.ErrorMessage + } + return "" +} + +func (x *RespondQueryTaskCompletedRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +type RespondQueryTaskCompletedResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RespondQueryTaskCompletedResponse) Reset() { + *x = RespondQueryTaskCompletedResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[67] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondQueryTaskCompletedResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondQueryTaskCompletedResponse) ProtoMessage() {} + +func (x *RespondQueryTaskCompletedResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[67] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondQueryTaskCompletedResponse.ProtoReflect.Descriptor instead. +func (*RespondQueryTaskCompletedResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{67} +} + +type ResetStickyTaskQueueRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + Execution *v13.WorkflowExecution `protobuf:"bytes,2,opt,name=execution,proto3" json:"execution,omitempty"` +} + +func (x *ResetStickyTaskQueueRequest) Reset() { + *x = ResetStickyTaskQueueRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[68] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResetStickyTaskQueueRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResetStickyTaskQueueRequest) ProtoMessage() {} + +func (x *ResetStickyTaskQueueRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[68] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResetStickyTaskQueueRequest.ProtoReflect.Descriptor instead. +func (*ResetStickyTaskQueueRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{68} +} + +func (x *ResetStickyTaskQueueRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ResetStickyTaskQueueRequest) GetExecution() *v13.WorkflowExecution { + if x != nil { + return x.Execution + } + return nil +} + +type ResetStickyTaskQueueResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *ResetStickyTaskQueueResponse) Reset() { + *x = ResetStickyTaskQueueResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[69] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResetStickyTaskQueueResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResetStickyTaskQueueResponse) ProtoMessage() {} + +func (x *ResetStickyTaskQueueResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[69] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResetStickyTaskQueueResponse.ProtoReflect.Descriptor instead. +func (*ResetStickyTaskQueueResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{69} +} + +type QueryWorkflowRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + Execution *v13.WorkflowExecution `protobuf:"bytes,2,opt,name=execution,proto3" json:"execution,omitempty"` + Query *v18.WorkflowQuery `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"` + // QueryRejectCondition can used to reject the query if workflow state does not satisfy condition. + // Default: QUERY_REJECT_CONDITION_NONE. + QueryRejectCondition v11.QueryRejectCondition `protobuf:"varint,4,opt,name=query_reject_condition,json=queryRejectCondition,proto3,enum=temporal.api.enums.v1.QueryRejectCondition" json:"query_reject_condition,omitempty"` +} + +func (x *QueryWorkflowRequest) Reset() { + *x = QueryWorkflowRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[70] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryWorkflowRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryWorkflowRequest) ProtoMessage() {} + +func (x *QueryWorkflowRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[70] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryWorkflowRequest.ProtoReflect.Descriptor instead. +func (*QueryWorkflowRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{70} +} + +func (x *QueryWorkflowRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *QueryWorkflowRequest) GetExecution() *v13.WorkflowExecution { + if x != nil { + return x.Execution + } + return nil +} + +func (x *QueryWorkflowRequest) GetQuery() *v18.WorkflowQuery { + if x != nil { + return x.Query + } + return nil +} + +func (x *QueryWorkflowRequest) GetQueryRejectCondition() v11.QueryRejectCondition { + if x != nil { + return x.QueryRejectCondition + } + return v11.QueryRejectCondition(0) +} + +type QueryWorkflowResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + QueryResult *v13.Payloads `protobuf:"bytes,1,opt,name=query_result,json=queryResult,proto3" json:"query_result,omitempty"` + QueryRejected *v18.QueryRejected `protobuf:"bytes,2,opt,name=query_rejected,json=queryRejected,proto3" json:"query_rejected,omitempty"` +} + +func (x *QueryWorkflowResponse) Reset() { + *x = QueryWorkflowResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[71] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryWorkflowResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryWorkflowResponse) ProtoMessage() {} + +func (x *QueryWorkflowResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[71] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QueryWorkflowResponse.ProtoReflect.Descriptor instead. +func (*QueryWorkflowResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{71} +} + +func (x *QueryWorkflowResponse) GetQueryResult() *v13.Payloads { + if x != nil { + return x.QueryResult + } + return nil +} + +func (x *QueryWorkflowResponse) GetQueryRejected() *v18.QueryRejected { + if x != nil { + return x.QueryRejected + } + return nil +} + +type DescribeWorkflowExecutionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + Execution *v13.WorkflowExecution `protobuf:"bytes,2,opt,name=execution,proto3" json:"execution,omitempty"` +} + +func (x *DescribeWorkflowExecutionRequest) Reset() { + *x = DescribeWorkflowExecutionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[72] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeWorkflowExecutionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeWorkflowExecutionRequest) ProtoMessage() {} + +func (x *DescribeWorkflowExecutionRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[72] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DescribeWorkflowExecutionRequest.ProtoReflect.Descriptor instead. +func (*DescribeWorkflowExecutionRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{72} +} + +func (x *DescribeWorkflowExecutionRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *DescribeWorkflowExecutionRequest) GetExecution() *v13.WorkflowExecution { + if x != nil { + return x.Execution + } + return nil +} + +type DescribeWorkflowExecutionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ExecutionConfig *v112.WorkflowExecutionConfig `protobuf:"bytes,1,opt,name=execution_config,json=executionConfig,proto3" json:"execution_config,omitempty"` + WorkflowExecutionInfo *v112.WorkflowExecutionInfo `protobuf:"bytes,2,opt,name=workflow_execution_info,json=workflowExecutionInfo,proto3" json:"workflow_execution_info,omitempty"` + PendingActivities []*v112.PendingActivityInfo `protobuf:"bytes,3,rep,name=pending_activities,json=pendingActivities,proto3" json:"pending_activities,omitempty"` + PendingChildren []*v112.PendingChildExecutionInfo `protobuf:"bytes,4,rep,name=pending_children,json=pendingChildren,proto3" json:"pending_children,omitempty"` + PendingWorkflowTask *v112.PendingWorkflowTaskInfo `protobuf:"bytes,5,opt,name=pending_workflow_task,json=pendingWorkflowTask,proto3" json:"pending_workflow_task,omitempty"` + Callbacks []*v112.CallbackInfo `protobuf:"bytes,6,rep,name=callbacks,proto3" json:"callbacks,omitempty"` + PendingNexusOperations []*v112.PendingNexusOperationInfo `protobuf:"bytes,7,rep,name=pending_nexus_operations,json=pendingNexusOperations,proto3" json:"pending_nexus_operations,omitempty"` +} + +func (x *DescribeWorkflowExecutionResponse) Reset() { + *x = DescribeWorkflowExecutionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[73] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeWorkflowExecutionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeWorkflowExecutionResponse) ProtoMessage() {} + +func (x *DescribeWorkflowExecutionResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[73] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DescribeWorkflowExecutionResponse.ProtoReflect.Descriptor instead. +func (*DescribeWorkflowExecutionResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{73} +} + +func (x *DescribeWorkflowExecutionResponse) GetExecutionConfig() *v112.WorkflowExecutionConfig { + if x != nil { + return x.ExecutionConfig + } + return nil +} + +func (x *DescribeWorkflowExecutionResponse) GetWorkflowExecutionInfo() *v112.WorkflowExecutionInfo { + if x != nil { + return x.WorkflowExecutionInfo + } + return nil +} + +func (x *DescribeWorkflowExecutionResponse) GetPendingActivities() []*v112.PendingActivityInfo { + if x != nil { + return x.PendingActivities + } + return nil +} + +func (x *DescribeWorkflowExecutionResponse) GetPendingChildren() []*v112.PendingChildExecutionInfo { + if x != nil { + return x.PendingChildren + } + return nil +} + +func (x *DescribeWorkflowExecutionResponse) GetPendingWorkflowTask() *v112.PendingWorkflowTaskInfo { + if x != nil { + return x.PendingWorkflowTask + } + return nil +} + +func (x *DescribeWorkflowExecutionResponse) GetCallbacks() []*v112.CallbackInfo { + if x != nil { + return x.Callbacks + } + return nil +} + +func (x *DescribeWorkflowExecutionResponse) GetPendingNexusOperations() []*v112.PendingNexusOperationInfo { + if x != nil { + return x.PendingNexusOperations + } + return nil +} + +// (-- api-linter: core::0203::optional=disabled +// +// aip.dev/not-precedent: field_behavior annotation not available in our gogo fork --) +type DescribeTaskQueueRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Sticky queues are not supported in `ENHANCED` mode. + TaskQueue *v14.TaskQueue `protobuf:"bytes,2,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + // Deprecated. Use `ENHANCED` mode with `task_queue_types`. Ignored in `ENHANCED` mode. + // If unspecified (TASK_QUEUE_TYPE_UNSPECIFIED), then default value (TASK_QUEUE_TYPE_WORKFLOW) will be used. + TaskQueueType v11.TaskQueueType `protobuf:"varint,3,opt,name=task_queue_type,json=taskQueueType,proto3,enum=temporal.api.enums.v1.TaskQueueType" json:"task_queue_type,omitempty"` + // Deprecated. Ignored in `ENHANCED` mode. + IncludeTaskQueueStatus bool `protobuf:"varint,4,opt,name=include_task_queue_status,json=includeTaskQueueStatus,proto3" json:"include_task_queue_status,omitempty"` + // All options except `task_queue_type` and `include_task_queue_status` are only available in the `ENHANCED` mode. + ApiMode v11.DescribeTaskQueueMode `protobuf:"varint,5,opt,name=api_mode,json=apiMode,proto3,enum=temporal.api.enums.v1.DescribeTaskQueueMode" json:"api_mode,omitempty"` + // Optional. If not provided, the result for the default Build ID will be returned. The default Build ID is the one + // mentioned in the first unconditional Assignment Rule. If there is no default Build ID, the result for the + // unversioned queue will be returned. + // (-- api-linter: core::0140::prepositions --) + Versions *v14.TaskQueueVersionSelection `protobuf:"bytes,6,opt,name=versions,proto3" json:"versions,omitempty"` + // Task queue types to report info about. If not specified, all types are considered. + TaskQueueTypes []v11.TaskQueueType `protobuf:"varint,7,rep,packed,name=task_queue_types,json=taskQueueTypes,proto3,enum=temporal.api.enums.v1.TaskQueueType" json:"task_queue_types,omitempty"` + // Report stats for the requested task queue types and versions + ReportStats bool `protobuf:"varint,8,opt,name=report_stats,json=reportStats,proto3" json:"report_stats,omitempty"` + // Report list of pollers for requested task queue types and versions + ReportPollers bool `protobuf:"varint,9,opt,name=report_pollers,json=reportPollers,proto3" json:"report_pollers,omitempty"` + // Report task reachability for the requested versions and all task types (task reachability is not reported + // per task type). + ReportTaskReachability bool `protobuf:"varint,10,opt,name=report_task_reachability,json=reportTaskReachability,proto3" json:"report_task_reachability,omitempty"` +} + +func (x *DescribeTaskQueueRequest) Reset() { + *x = DescribeTaskQueueRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[74] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeTaskQueueRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeTaskQueueRequest) ProtoMessage() {} + +func (x *DescribeTaskQueueRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[74] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DescribeTaskQueueRequest.ProtoReflect.Descriptor instead. +func (*DescribeTaskQueueRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{74} +} + +func (x *DescribeTaskQueueRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *DescribeTaskQueueRequest) GetTaskQueue() *v14.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *DescribeTaskQueueRequest) GetTaskQueueType() v11.TaskQueueType { + if x != nil { + return x.TaskQueueType + } + return v11.TaskQueueType(0) +} + +func (x *DescribeTaskQueueRequest) GetIncludeTaskQueueStatus() bool { + if x != nil { + return x.IncludeTaskQueueStatus + } + return false +} + +func (x *DescribeTaskQueueRequest) GetApiMode() v11.DescribeTaskQueueMode { + if x != nil { + return x.ApiMode + } + return v11.DescribeTaskQueueMode(0) +} + +func (x *DescribeTaskQueueRequest) GetVersions() *v14.TaskQueueVersionSelection { + if x != nil { + return x.Versions + } + return nil +} + +func (x *DescribeTaskQueueRequest) GetTaskQueueTypes() []v11.TaskQueueType { + if x != nil { + return x.TaskQueueTypes + } + return nil +} + +func (x *DescribeTaskQueueRequest) GetReportStats() bool { + if x != nil { + return x.ReportStats + } + return false +} + +func (x *DescribeTaskQueueRequest) GetReportPollers() bool { + if x != nil { + return x.ReportPollers + } + return false +} + +func (x *DescribeTaskQueueRequest) GetReportTaskReachability() bool { + if x != nil { + return x.ReportTaskReachability + } + return false +} + +type DescribeTaskQueueResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Deprecated. Use `versions_info.types_info.pollers` with `ENHANCED` mode instead. + // Not set in `ENHANCED` mode. + Pollers []*v14.PollerInfo `protobuf:"bytes,1,rep,name=pollers,proto3" json:"pollers,omitempty"` + // Deprecated. Not set in `ENHANCED` mode. + TaskQueueStatus *v14.TaskQueueStatus `protobuf:"bytes,2,opt,name=task_queue_status,json=taskQueueStatus,proto3" json:"task_queue_status,omitempty"` + // This map contains Task Queue information for each Build ID. Empty string as key value means unversioned. + // Only set in `ENHANCED` mode. + VersionsInfo map[string]*v14.TaskQueueVersionInfo `protobuf:"bytes,3,rep,name=versions_info,json=versionsInfo,proto3" json:"versions_info,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *DescribeTaskQueueResponse) Reset() { + *x = DescribeTaskQueueResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[75] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeTaskQueueResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeTaskQueueResponse) ProtoMessage() {} + +func (x *DescribeTaskQueueResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[75] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DescribeTaskQueueResponse.ProtoReflect.Descriptor instead. +func (*DescribeTaskQueueResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{75} +} + +func (x *DescribeTaskQueueResponse) GetPollers() []*v14.PollerInfo { + if x != nil { + return x.Pollers + } + return nil +} + +func (x *DescribeTaskQueueResponse) GetTaskQueueStatus() *v14.TaskQueueStatus { + if x != nil { + return x.TaskQueueStatus + } + return nil +} + +func (x *DescribeTaskQueueResponse) GetVersionsInfo() map[string]*v14.TaskQueueVersionInfo { + if x != nil { + return x.VersionsInfo + } + return nil +} + +type GetClusterInfoRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *GetClusterInfoRequest) Reset() { + *x = GetClusterInfoRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[76] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetClusterInfoRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetClusterInfoRequest) ProtoMessage() {} + +func (x *GetClusterInfoRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[76] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetClusterInfoRequest.ProtoReflect.Descriptor instead. +func (*GetClusterInfoRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{76} +} + +// GetClusterInfoResponse contains information about Temporal cluster. +type GetClusterInfoResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Key is client name i.e "temporal-go", "temporal-java", or "temporal-cli". + // Value is ranges of supported versions of this client i.e ">1.1.1 <=1.4.0 || ^5.0.0". + SupportedClients map[string]string `protobuf:"bytes,1,rep,name=supported_clients,json=supportedClients,proto3" json:"supported_clients,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + ServerVersion string `protobuf:"bytes,2,opt,name=server_version,json=serverVersion,proto3" json:"server_version,omitempty"` + ClusterId string `protobuf:"bytes,3,opt,name=cluster_id,json=clusterId,proto3" json:"cluster_id,omitempty"` + VersionInfo *v113.VersionInfo `protobuf:"bytes,4,opt,name=version_info,json=versionInfo,proto3" json:"version_info,omitempty"` + ClusterName string `protobuf:"bytes,5,opt,name=cluster_name,json=clusterName,proto3" json:"cluster_name,omitempty"` + HistoryShardCount int32 `protobuf:"varint,6,opt,name=history_shard_count,json=historyShardCount,proto3" json:"history_shard_count,omitempty"` + PersistenceStore string `protobuf:"bytes,7,opt,name=persistence_store,json=persistenceStore,proto3" json:"persistence_store,omitempty"` + VisibilityStore string `protobuf:"bytes,8,opt,name=visibility_store,json=visibilityStore,proto3" json:"visibility_store,omitempty"` +} + +func (x *GetClusterInfoResponse) Reset() { + *x = GetClusterInfoResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[77] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetClusterInfoResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetClusterInfoResponse) ProtoMessage() {} + +func (x *GetClusterInfoResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[77] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetClusterInfoResponse.ProtoReflect.Descriptor instead. +func (*GetClusterInfoResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{77} +} + +func (x *GetClusterInfoResponse) GetSupportedClients() map[string]string { + if x != nil { + return x.SupportedClients + } + return nil +} + +func (x *GetClusterInfoResponse) GetServerVersion() string { + if x != nil { + return x.ServerVersion + } + return "" +} + +func (x *GetClusterInfoResponse) GetClusterId() string { + if x != nil { + return x.ClusterId + } + return "" +} + +func (x *GetClusterInfoResponse) GetVersionInfo() *v113.VersionInfo { + if x != nil { + return x.VersionInfo + } + return nil +} + +func (x *GetClusterInfoResponse) GetClusterName() string { + if x != nil { + return x.ClusterName + } + return "" +} + +func (x *GetClusterInfoResponse) GetHistoryShardCount() int32 { + if x != nil { + return x.HistoryShardCount + } + return 0 +} + +func (x *GetClusterInfoResponse) GetPersistenceStore() string { + if x != nil { + return x.PersistenceStore + } + return "" +} + +func (x *GetClusterInfoResponse) GetVisibilityStore() string { + if x != nil { + return x.VisibilityStore + } + return "" +} + +type GetSystemInfoRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *GetSystemInfoRequest) Reset() { + *x = GetSystemInfoRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[78] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSystemInfoRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSystemInfoRequest) ProtoMessage() {} + +func (x *GetSystemInfoRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[78] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetSystemInfoRequest.ProtoReflect.Descriptor instead. +func (*GetSystemInfoRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{78} +} + +type GetSystemInfoResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Version of the server. + ServerVersion string `protobuf:"bytes,1,opt,name=server_version,json=serverVersion,proto3" json:"server_version,omitempty"` + // All capabilities the system supports. + Capabilities *GetSystemInfoResponse_Capabilities `protobuf:"bytes,2,opt,name=capabilities,proto3" json:"capabilities,omitempty"` +} + +func (x *GetSystemInfoResponse) Reset() { + *x = GetSystemInfoResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[79] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSystemInfoResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSystemInfoResponse) ProtoMessage() {} + +func (x *GetSystemInfoResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[79] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetSystemInfoResponse.ProtoReflect.Descriptor instead. +func (*GetSystemInfoResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{79} +} + +func (x *GetSystemInfoResponse) GetServerVersion() string { + if x != nil { + return x.ServerVersion + } + return "" +} + +func (x *GetSystemInfoResponse) GetCapabilities() *GetSystemInfoResponse_Capabilities { + if x != nil { + return x.Capabilities + } + return nil +} + +type ListTaskQueuePartitionsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + TaskQueue *v14.TaskQueue `protobuf:"bytes,2,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` +} + +func (x *ListTaskQueuePartitionsRequest) Reset() { + *x = ListTaskQueuePartitionsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[80] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListTaskQueuePartitionsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListTaskQueuePartitionsRequest) ProtoMessage() {} + +func (x *ListTaskQueuePartitionsRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[80] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListTaskQueuePartitionsRequest.ProtoReflect.Descriptor instead. +func (*ListTaskQueuePartitionsRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{80} +} + +func (x *ListTaskQueuePartitionsRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ListTaskQueuePartitionsRequest) GetTaskQueue() *v14.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +type ListTaskQueuePartitionsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ActivityTaskQueuePartitions []*v14.TaskQueuePartitionMetadata `protobuf:"bytes,1,rep,name=activity_task_queue_partitions,json=activityTaskQueuePartitions,proto3" json:"activity_task_queue_partitions,omitempty"` + WorkflowTaskQueuePartitions []*v14.TaskQueuePartitionMetadata `protobuf:"bytes,2,rep,name=workflow_task_queue_partitions,json=workflowTaskQueuePartitions,proto3" json:"workflow_task_queue_partitions,omitempty"` +} + +func (x *ListTaskQueuePartitionsResponse) Reset() { + *x = ListTaskQueuePartitionsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[81] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListTaskQueuePartitionsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListTaskQueuePartitionsResponse) ProtoMessage() {} + +func (x *ListTaskQueuePartitionsResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[81] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListTaskQueuePartitionsResponse.ProtoReflect.Descriptor instead. +func (*ListTaskQueuePartitionsResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{81} +} + +func (x *ListTaskQueuePartitionsResponse) GetActivityTaskQueuePartitions() []*v14.TaskQueuePartitionMetadata { + if x != nil { + return x.ActivityTaskQueuePartitions + } + return nil +} + +func (x *ListTaskQueuePartitionsResponse) GetWorkflowTaskQueuePartitions() []*v14.TaskQueuePartitionMetadata { + if x != nil { + return x.WorkflowTaskQueuePartitions + } + return nil +} + +// (-- api-linter: core::0203::optional=disabled +// +// aip.dev/not-precedent: field_behavior annotation not available in our gogo fork --) +type CreateScheduleRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace the schedule should be created in. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The id of the new schedule. + ScheduleId string `protobuf:"bytes,2,opt,name=schedule_id,json=scheduleId,proto3" json:"schedule_id,omitempty"` + // The schedule spec, policies, action, and initial state. + Schedule *v114.Schedule `protobuf:"bytes,3,opt,name=schedule,proto3" json:"schedule,omitempty"` + // Optional initial patch (e.g. to run the action once immediately). + InitialPatch *v114.SchedulePatch `protobuf:"bytes,4,opt,name=initial_patch,json=initialPatch,proto3" json:"initial_patch,omitempty"` + // The identity of the client who initiated this request. + Identity string `protobuf:"bytes,5,opt,name=identity,proto3" json:"identity,omitempty"` + // A unique identifier for this create request for idempotence. Typically UUIDv4. + RequestId string `protobuf:"bytes,6,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` + // Memo and search attributes to attach to the schedule itself. + Memo *v13.Memo `protobuf:"bytes,7,opt,name=memo,proto3" json:"memo,omitempty"` + SearchAttributes *v13.SearchAttributes `protobuf:"bytes,8,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` +} + +func (x *CreateScheduleRequest) Reset() { + *x = CreateScheduleRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[82] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateScheduleRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateScheduleRequest) ProtoMessage() {} + +func (x *CreateScheduleRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[82] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateScheduleRequest.ProtoReflect.Descriptor instead. +func (*CreateScheduleRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{82} +} + +func (x *CreateScheduleRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *CreateScheduleRequest) GetScheduleId() string { + if x != nil { + return x.ScheduleId + } + return "" +} + +func (x *CreateScheduleRequest) GetSchedule() *v114.Schedule { + if x != nil { + return x.Schedule + } + return nil +} + +func (x *CreateScheduleRequest) GetInitialPatch() *v114.SchedulePatch { + if x != nil { + return x.InitialPatch + } + return nil +} + +func (x *CreateScheduleRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *CreateScheduleRequest) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +func (x *CreateScheduleRequest) GetMemo() *v13.Memo { + if x != nil { + return x.Memo + } + return nil +} + +func (x *CreateScheduleRequest) GetSearchAttributes() *v13.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +type CreateScheduleResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ConflictToken []byte `protobuf:"bytes,1,opt,name=conflict_token,json=conflictToken,proto3" json:"conflict_token,omitempty"` +} + +func (x *CreateScheduleResponse) Reset() { + *x = CreateScheduleResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[83] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateScheduleResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateScheduleResponse) ProtoMessage() {} + +func (x *CreateScheduleResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[83] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateScheduleResponse.ProtoReflect.Descriptor instead. +func (*CreateScheduleResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{83} +} + +func (x *CreateScheduleResponse) GetConflictToken() []byte { + if x != nil { + return x.ConflictToken + } + return nil +} + +type DescribeScheduleRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace of the schedule to describe. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The id of the schedule to describe. + ScheduleId string `protobuf:"bytes,2,opt,name=schedule_id,json=scheduleId,proto3" json:"schedule_id,omitempty"` +} + +func (x *DescribeScheduleRequest) Reset() { + *x = DescribeScheduleRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[84] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeScheduleRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeScheduleRequest) ProtoMessage() {} + +func (x *DescribeScheduleRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[84] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DescribeScheduleRequest.ProtoReflect.Descriptor instead. +func (*DescribeScheduleRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{84} +} + +func (x *DescribeScheduleRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *DescribeScheduleRequest) GetScheduleId() string { + if x != nil { + return x.ScheduleId + } + return "" +} + +type DescribeScheduleResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The complete current schedule details. This may not match the schedule as + // created because: + // - some types of schedule specs may get compiled into others (e.g. + // CronString into StructuredCalendarSpec) + // - some unspecified fields may be replaced by defaults + // - some fields in the state are modified automatically + // - the schedule may have been modified by UpdateSchedule or PatchSchedule + Schedule *v114.Schedule `protobuf:"bytes,1,opt,name=schedule,proto3" json:"schedule,omitempty"` + // Extra schedule state info. + Info *v114.ScheduleInfo `protobuf:"bytes,2,opt,name=info,proto3" json:"info,omitempty"` + // The memo and search attributes that the schedule was created with. + Memo *v13.Memo `protobuf:"bytes,3,opt,name=memo,proto3" json:"memo,omitempty"` + SearchAttributes *v13.SearchAttributes `protobuf:"bytes,4,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` + // This value can be passed back to UpdateSchedule to ensure that the + // schedule was not modified between a Describe and an Update, which could + // lead to lost updates and other confusion. + ConflictToken []byte `protobuf:"bytes,5,opt,name=conflict_token,json=conflictToken,proto3" json:"conflict_token,omitempty"` +} + +func (x *DescribeScheduleResponse) Reset() { + *x = DescribeScheduleResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[85] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeScheduleResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeScheduleResponse) ProtoMessage() {} + +func (x *DescribeScheduleResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[85] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DescribeScheduleResponse.ProtoReflect.Descriptor instead. +func (*DescribeScheduleResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{85} +} + +func (x *DescribeScheduleResponse) GetSchedule() *v114.Schedule { + if x != nil { + return x.Schedule + } + return nil +} + +func (x *DescribeScheduleResponse) GetInfo() *v114.ScheduleInfo { + if x != nil { + return x.Info + } + return nil +} + +func (x *DescribeScheduleResponse) GetMemo() *v13.Memo { + if x != nil { + return x.Memo + } + return nil +} + +func (x *DescribeScheduleResponse) GetSearchAttributes() *v13.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +func (x *DescribeScheduleResponse) GetConflictToken() []byte { + if x != nil { + return x.ConflictToken + } + return nil +} + +type UpdateScheduleRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace of the schedule to update. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The id of the schedule to update. + ScheduleId string `protobuf:"bytes,2,opt,name=schedule_id,json=scheduleId,proto3" json:"schedule_id,omitempty"` + // The new schedule. The four main fields of the schedule (spec, action, + // policies, state) are replaced completely by the values in this message. + Schedule *v114.Schedule `protobuf:"bytes,3,opt,name=schedule,proto3" json:"schedule,omitempty"` + // This can be the value of conflict_token from a DescribeScheduleResponse, + // which will cause this request to fail if the schedule has been modified + // between the Describe and this Update. + // If missing, the schedule will be updated unconditionally. + ConflictToken []byte `protobuf:"bytes,4,opt,name=conflict_token,json=conflictToken,proto3" json:"conflict_token,omitempty"` + // The identity of the client who initiated this request. + Identity string `protobuf:"bytes,5,opt,name=identity,proto3" json:"identity,omitempty"` + // A unique identifier for this update request for idempotence. Typically UUIDv4. + RequestId string `protobuf:"bytes,6,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` + // Schedule search attributes to be updated. + // Do not set this field if you do not want to update the search attributes. + // A non-null empty object will set the search attributes to an empty map. + // Note: you cannot only update the search attributes with `UpdateScheduleRequest`, + // you must also set the `schedule` field; otherwise, it will unset the schedule. + SearchAttributes *v13.SearchAttributes `protobuf:"bytes,7,opt,name=search_attributes,json=searchAttributes,proto3" json:"search_attributes,omitempty"` +} + +func (x *UpdateScheduleRequest) Reset() { + *x = UpdateScheduleRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[86] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateScheduleRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateScheduleRequest) ProtoMessage() {} + +func (x *UpdateScheduleRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[86] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateScheduleRequest.ProtoReflect.Descriptor instead. +func (*UpdateScheduleRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{86} +} + +func (x *UpdateScheduleRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *UpdateScheduleRequest) GetScheduleId() string { + if x != nil { + return x.ScheduleId + } + return "" +} + +func (x *UpdateScheduleRequest) GetSchedule() *v114.Schedule { + if x != nil { + return x.Schedule + } + return nil +} + +func (x *UpdateScheduleRequest) GetConflictToken() []byte { + if x != nil { + return x.ConflictToken + } + return nil +} + +func (x *UpdateScheduleRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *UpdateScheduleRequest) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +func (x *UpdateScheduleRequest) GetSearchAttributes() *v13.SearchAttributes { + if x != nil { + return x.SearchAttributes + } + return nil +} + +type UpdateScheduleResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *UpdateScheduleResponse) Reset() { + *x = UpdateScheduleResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[87] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateScheduleResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateScheduleResponse) ProtoMessage() {} + +func (x *UpdateScheduleResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[87] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateScheduleResponse.ProtoReflect.Descriptor instead. +func (*UpdateScheduleResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{87} +} + +type PatchScheduleRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace of the schedule to patch. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The id of the schedule to patch. + ScheduleId string `protobuf:"bytes,2,opt,name=schedule_id,json=scheduleId,proto3" json:"schedule_id,omitempty"` + Patch *v114.SchedulePatch `protobuf:"bytes,3,opt,name=patch,proto3" json:"patch,omitempty"` + // The identity of the client who initiated this request. + Identity string `protobuf:"bytes,4,opt,name=identity,proto3" json:"identity,omitempty"` + // A unique identifier for this update request for idempotence. Typically UUIDv4. + RequestId string `protobuf:"bytes,5,opt,name=request_id,json=requestId,proto3" json:"request_id,omitempty"` +} + +func (x *PatchScheduleRequest) Reset() { + *x = PatchScheduleRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[88] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PatchScheduleRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PatchScheduleRequest) ProtoMessage() {} + +func (x *PatchScheduleRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[88] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PatchScheduleRequest.ProtoReflect.Descriptor instead. +func (*PatchScheduleRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{88} +} + +func (x *PatchScheduleRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *PatchScheduleRequest) GetScheduleId() string { + if x != nil { + return x.ScheduleId + } + return "" +} + +func (x *PatchScheduleRequest) GetPatch() *v114.SchedulePatch { + if x != nil { + return x.Patch + } + return nil +} + +func (x *PatchScheduleRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *PatchScheduleRequest) GetRequestId() string { + if x != nil { + return x.RequestId + } + return "" +} + +type PatchScheduleResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *PatchScheduleResponse) Reset() { + *x = PatchScheduleResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[89] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PatchScheduleResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PatchScheduleResponse) ProtoMessage() {} + +func (x *PatchScheduleResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[89] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PatchScheduleResponse.ProtoReflect.Descriptor instead. +func (*PatchScheduleResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{89} +} + +type ListScheduleMatchingTimesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace of the schedule to query. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The id of the schedule to query. + ScheduleId string `protobuf:"bytes,2,opt,name=schedule_id,json=scheduleId,proto3" json:"schedule_id,omitempty"` + // Time range to query. + StartTime *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + EndTime *timestamppb.Timestamp `protobuf:"bytes,4,opt,name=end_time,json=endTime,proto3" json:"end_time,omitempty"` +} + +func (x *ListScheduleMatchingTimesRequest) Reset() { + *x = ListScheduleMatchingTimesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[90] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListScheduleMatchingTimesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListScheduleMatchingTimesRequest) ProtoMessage() {} + +func (x *ListScheduleMatchingTimesRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[90] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListScheduleMatchingTimesRequest.ProtoReflect.Descriptor instead. +func (*ListScheduleMatchingTimesRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{90} +} + +func (x *ListScheduleMatchingTimesRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ListScheduleMatchingTimesRequest) GetScheduleId() string { + if x != nil { + return x.ScheduleId + } + return "" +} + +func (x *ListScheduleMatchingTimesRequest) GetStartTime() *timestamppb.Timestamp { + if x != nil { + return x.StartTime + } + return nil +} + +func (x *ListScheduleMatchingTimesRequest) GetEndTime() *timestamppb.Timestamp { + if x != nil { + return x.EndTime + } + return nil +} + +type ListScheduleMatchingTimesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + StartTime []*timestamppb.Timestamp `protobuf:"bytes,1,rep,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` +} + +func (x *ListScheduleMatchingTimesResponse) Reset() { + *x = ListScheduleMatchingTimesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[91] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListScheduleMatchingTimesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListScheduleMatchingTimesResponse) ProtoMessage() {} + +func (x *ListScheduleMatchingTimesResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[91] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListScheduleMatchingTimesResponse.ProtoReflect.Descriptor instead. +func (*ListScheduleMatchingTimesResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{91} +} + +func (x *ListScheduleMatchingTimesResponse) GetStartTime() []*timestamppb.Timestamp { + if x != nil { + return x.StartTime + } + return nil +} + +type DeleteScheduleRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace of the schedule to delete. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The id of the schedule to delete. + ScheduleId string `protobuf:"bytes,2,opt,name=schedule_id,json=scheduleId,proto3" json:"schedule_id,omitempty"` + // The identity of the client who initiated this request. + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` +} + +func (x *DeleteScheduleRequest) Reset() { + *x = DeleteScheduleRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[92] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteScheduleRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteScheduleRequest) ProtoMessage() {} + +func (x *DeleteScheduleRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[92] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteScheduleRequest.ProtoReflect.Descriptor instead. +func (*DeleteScheduleRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{92} +} + +func (x *DeleteScheduleRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *DeleteScheduleRequest) GetScheduleId() string { + if x != nil { + return x.ScheduleId + } + return "" +} + +func (x *DeleteScheduleRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +type DeleteScheduleResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *DeleteScheduleResponse) Reset() { + *x = DeleteScheduleResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[93] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteScheduleResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteScheduleResponse) ProtoMessage() {} + +func (x *DeleteScheduleResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[93] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteScheduleResponse.ProtoReflect.Descriptor instead. +func (*DeleteScheduleResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{93} +} + +type ListSchedulesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace to list schedules in. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // How many to return at once. + MaximumPageSize int32 `protobuf:"varint,2,opt,name=maximum_page_size,json=maximumPageSize,proto3" json:"maximum_page_size,omitempty"` + // Token to get the next page of results. + NextPageToken []byte `protobuf:"bytes,3,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` + // Query to filter schedules. + Query string `protobuf:"bytes,4,opt,name=query,proto3" json:"query,omitempty"` +} + +func (x *ListSchedulesRequest) Reset() { + *x = ListSchedulesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[94] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListSchedulesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListSchedulesRequest) ProtoMessage() {} + +func (x *ListSchedulesRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[94] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListSchedulesRequest.ProtoReflect.Descriptor instead. +func (*ListSchedulesRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{94} +} + +func (x *ListSchedulesRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ListSchedulesRequest) GetMaximumPageSize() int32 { + if x != nil { + return x.MaximumPageSize + } + return 0 +} + +func (x *ListSchedulesRequest) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +func (x *ListSchedulesRequest) GetQuery() string { + if x != nil { + return x.Query + } + return "" +} + +type ListSchedulesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Schedules []*v114.ScheduleListEntry `protobuf:"bytes,1,rep,name=schedules,proto3" json:"schedules,omitempty"` + NextPageToken []byte `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *ListSchedulesResponse) Reset() { + *x = ListSchedulesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[95] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListSchedulesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListSchedulesResponse) ProtoMessage() {} + +func (x *ListSchedulesResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[95] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListSchedulesResponse.ProtoReflect.Descriptor instead. +func (*ListSchedulesResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{95} +} + +func (x *ListSchedulesResponse) GetSchedules() []*v114.ScheduleListEntry { + if x != nil { + return x.Schedules + } + return nil +} + +func (x *ListSchedulesResponse) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +type UpdateWorkerBuildIdCompatibilityRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Must be set, the task queue to apply changes to. Because all workers on a given task queue + // must have the same set of workflow & activity implementations, there is no reason to specify + // a task queue type here. + TaskQueue string `protobuf:"bytes,2,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + // Types that are assignable to Operation: + // + // *UpdateWorkerBuildIdCompatibilityRequest_AddNewBuildIdInNewDefaultSet + // *UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleBuildId + // *UpdateWorkerBuildIdCompatibilityRequest_PromoteSetByBuildId + // *UpdateWorkerBuildIdCompatibilityRequest_PromoteBuildIdWithinSet + // *UpdateWorkerBuildIdCompatibilityRequest_MergeSets_ + Operation isUpdateWorkerBuildIdCompatibilityRequest_Operation `protobuf_oneof:"operation"` +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest) Reset() { + *x = UpdateWorkerBuildIdCompatibilityRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[96] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkerBuildIdCompatibilityRequest) ProtoMessage() {} + +func (x *UpdateWorkerBuildIdCompatibilityRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[96] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkerBuildIdCompatibilityRequest.ProtoReflect.Descriptor instead. +func (*UpdateWorkerBuildIdCompatibilityRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{96} +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest) GetTaskQueue() string { + if x != nil { + return x.TaskQueue + } + return "" +} + +func (m *UpdateWorkerBuildIdCompatibilityRequest) GetOperation() isUpdateWorkerBuildIdCompatibilityRequest_Operation { + if m != nil { + return m.Operation + } + return nil +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest) GetAddNewBuildIdInNewDefaultSet() string { + if x, ok := x.GetOperation().(*UpdateWorkerBuildIdCompatibilityRequest_AddNewBuildIdInNewDefaultSet); ok { + return x.AddNewBuildIdInNewDefaultSet + } + return "" +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest) GetAddNewCompatibleBuildId() *UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion { + if x, ok := x.GetOperation().(*UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleBuildId); ok { + return x.AddNewCompatibleBuildId + } + return nil +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest) GetPromoteSetByBuildId() string { + if x, ok := x.GetOperation().(*UpdateWorkerBuildIdCompatibilityRequest_PromoteSetByBuildId); ok { + return x.PromoteSetByBuildId + } + return "" +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest) GetPromoteBuildIdWithinSet() string { + if x, ok := x.GetOperation().(*UpdateWorkerBuildIdCompatibilityRequest_PromoteBuildIdWithinSet); ok { + return x.PromoteBuildIdWithinSet + } + return "" +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest) GetMergeSets() *UpdateWorkerBuildIdCompatibilityRequest_MergeSets { + if x, ok := x.GetOperation().(*UpdateWorkerBuildIdCompatibilityRequest_MergeSets_); ok { + return x.MergeSets + } + return nil +} + +type isUpdateWorkerBuildIdCompatibilityRequest_Operation interface { + isUpdateWorkerBuildIdCompatibilityRequest_Operation() +} + +type UpdateWorkerBuildIdCompatibilityRequest_AddNewBuildIdInNewDefaultSet struct { + // A new build id. This operation will create a new set which will be the new overall + // default version for the queue, with this id as its only member. This new set is + // incompatible with all previous sets/versions. + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: In makes perfect sense here. --) + AddNewBuildIdInNewDefaultSet string `protobuf:"bytes,3,opt,name=add_new_build_id_in_new_default_set,json=addNewBuildIdInNewDefaultSet,proto3,oneof"` +} + +type UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleBuildId struct { + // Adds a new id to an existing compatible set, see sub-message definition for more. + AddNewCompatibleBuildId *UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion `protobuf:"bytes,4,opt,name=add_new_compatible_build_id,json=addNewCompatibleBuildId,proto3,oneof"` +} + +type UpdateWorkerBuildIdCompatibilityRequest_PromoteSetByBuildId struct { + // Promote an existing set to be the current default (if it isn't already) by targeting + // an existing build id within it. This field's value is the extant build id. + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: Names are hard. --) + PromoteSetByBuildId string `protobuf:"bytes,5,opt,name=promote_set_by_build_id,json=promoteSetByBuildId,proto3,oneof"` +} + +type UpdateWorkerBuildIdCompatibilityRequest_PromoteBuildIdWithinSet struct { + // Promote an existing build id within some set to be the current default for that set. + // + // (-- api-linter: core::0140::prepositions=disabled + // + // aip.dev/not-precedent: Within makes perfect sense here. --) + PromoteBuildIdWithinSet string `protobuf:"bytes,6,opt,name=promote_build_id_within_set,json=promoteBuildIdWithinSet,proto3,oneof"` +} + +type UpdateWorkerBuildIdCompatibilityRequest_MergeSets_ struct { + // Merge two existing sets together, thus declaring all build IDs in both sets compatible + // with one another. The primary set's default will become the default for the merged set. + // This is useful if you've accidentally declared a new ID as incompatible you meant to + // declare as compatible. The unusual case of incomplete replication during failover could + // also result in a split set, which this operation can repair. + MergeSets *UpdateWorkerBuildIdCompatibilityRequest_MergeSets `protobuf:"bytes,7,opt,name=merge_sets,json=mergeSets,proto3,oneof"` +} + +func (*UpdateWorkerBuildIdCompatibilityRequest_AddNewBuildIdInNewDefaultSet) isUpdateWorkerBuildIdCompatibilityRequest_Operation() { +} + +func (*UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleBuildId) isUpdateWorkerBuildIdCompatibilityRequest_Operation() { +} + +func (*UpdateWorkerBuildIdCompatibilityRequest_PromoteSetByBuildId) isUpdateWorkerBuildIdCompatibilityRequest_Operation() { +} + +func (*UpdateWorkerBuildIdCompatibilityRequest_PromoteBuildIdWithinSet) isUpdateWorkerBuildIdCompatibilityRequest_Operation() { +} + +func (*UpdateWorkerBuildIdCompatibilityRequest_MergeSets_) isUpdateWorkerBuildIdCompatibilityRequest_Operation() { +} + +type UpdateWorkerBuildIdCompatibilityResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *UpdateWorkerBuildIdCompatibilityResponse) Reset() { + *x = UpdateWorkerBuildIdCompatibilityResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[97] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkerBuildIdCompatibilityResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkerBuildIdCompatibilityResponse) ProtoMessage() {} + +func (x *UpdateWorkerBuildIdCompatibilityResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[97] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkerBuildIdCompatibilityResponse.ProtoReflect.Descriptor instead. +func (*UpdateWorkerBuildIdCompatibilityResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{97} +} + +type GetWorkerBuildIdCompatibilityRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Must be set, the task queue to interrogate about worker id compatibility. + TaskQueue string `protobuf:"bytes,2,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + // Limits how many compatible sets will be returned. Specify 1 to only return the current + // default major version set. 0 returns all sets. + MaxSets int32 `protobuf:"varint,3,opt,name=max_sets,json=maxSets,proto3" json:"max_sets,omitempty"` +} + +func (x *GetWorkerBuildIdCompatibilityRequest) Reset() { + *x = GetWorkerBuildIdCompatibilityRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[98] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetWorkerBuildIdCompatibilityRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetWorkerBuildIdCompatibilityRequest) ProtoMessage() {} + +func (x *GetWorkerBuildIdCompatibilityRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[98] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetWorkerBuildIdCompatibilityRequest.ProtoReflect.Descriptor instead. +func (*GetWorkerBuildIdCompatibilityRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{98} +} + +func (x *GetWorkerBuildIdCompatibilityRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *GetWorkerBuildIdCompatibilityRequest) GetTaskQueue() string { + if x != nil { + return x.TaskQueue + } + return "" +} + +func (x *GetWorkerBuildIdCompatibilityRequest) GetMaxSets() int32 { + if x != nil { + return x.MaxSets + } + return 0 +} + +type GetWorkerBuildIdCompatibilityResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Major version sets, in order from oldest to newest. The last element of the list will always + // be the current default major version. IE: New workflows will target the most recent version + // in that version set. + // + // There may be fewer sets returned than exist, if the request chose to limit this response. + MajorVersionSets []*v14.CompatibleVersionSet `protobuf:"bytes,1,rep,name=major_version_sets,json=majorVersionSets,proto3" json:"major_version_sets,omitempty"` +} + +func (x *GetWorkerBuildIdCompatibilityResponse) Reset() { + *x = GetWorkerBuildIdCompatibilityResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[99] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetWorkerBuildIdCompatibilityResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetWorkerBuildIdCompatibilityResponse) ProtoMessage() {} + +func (x *GetWorkerBuildIdCompatibilityResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[99] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetWorkerBuildIdCompatibilityResponse.ProtoReflect.Descriptor instead. +func (*GetWorkerBuildIdCompatibilityResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{99} +} + +func (x *GetWorkerBuildIdCompatibilityResponse) GetMajorVersionSets() []*v14.CompatibleVersionSet { + if x != nil { + return x.MajorVersionSets + } + return nil +} + +// (-- api-linter: core::0134::request-mask-required=disabled +// +// aip.dev/not-precedent: UpdateNamespace RPC doesn't follow Google API format. --) +// +// (-- api-linter: core::0134::request-resource-required=disabled +// +// aip.dev/not-precedent: GetWorkerBuildIdCompatibilityRequest RPC doesn't follow Google API format. --) +type UpdateWorkerVersioningRulesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + TaskQueue string `protobuf:"bytes,2,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + // A valid conflict_token can be taken from the previous + // ListWorkerVersioningRulesResponse or UpdateWorkerVersioningRulesResponse. + // An invalid token will cause this request to fail, ensuring that if the rules + // for this Task Queue have been modified between the previous and current + // operation, the request will fail instead of causing an unpredictable mutation. + ConflictToken []byte `protobuf:"bytes,3,opt,name=conflict_token,json=conflictToken,proto3" json:"conflict_token,omitempty"` + // Types that are assignable to Operation: + // + // *UpdateWorkerVersioningRulesRequest_InsertAssignmentRule + // *UpdateWorkerVersioningRulesRequest_ReplaceAssignmentRule + // *UpdateWorkerVersioningRulesRequest_DeleteAssignmentRule + // *UpdateWorkerVersioningRulesRequest_AddCompatibleRedirectRule + // *UpdateWorkerVersioningRulesRequest_ReplaceCompatibleRedirectRule + // *UpdateWorkerVersioningRulesRequest_DeleteCompatibleRedirectRule + // *UpdateWorkerVersioningRulesRequest_CommitBuildId_ + Operation isUpdateWorkerVersioningRulesRequest_Operation `protobuf_oneof:"operation"` +} + +func (x *UpdateWorkerVersioningRulesRequest) Reset() { + *x = UpdateWorkerVersioningRulesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[100] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkerVersioningRulesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkerVersioningRulesRequest) ProtoMessage() {} + +func (x *UpdateWorkerVersioningRulesRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[100] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkerVersioningRulesRequest.ProtoReflect.Descriptor instead. +func (*UpdateWorkerVersioningRulesRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{100} +} + +func (x *UpdateWorkerVersioningRulesRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *UpdateWorkerVersioningRulesRequest) GetTaskQueue() string { + if x != nil { + return x.TaskQueue + } + return "" +} + +func (x *UpdateWorkerVersioningRulesRequest) GetConflictToken() []byte { + if x != nil { + return x.ConflictToken + } + return nil +} + +func (m *UpdateWorkerVersioningRulesRequest) GetOperation() isUpdateWorkerVersioningRulesRequest_Operation { + if m != nil { + return m.Operation + } + return nil +} + +func (x *UpdateWorkerVersioningRulesRequest) GetInsertAssignmentRule() *UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule { + if x, ok := x.GetOperation().(*UpdateWorkerVersioningRulesRequest_InsertAssignmentRule); ok { + return x.InsertAssignmentRule + } + return nil +} + +func (x *UpdateWorkerVersioningRulesRequest) GetReplaceAssignmentRule() *UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule { + if x, ok := x.GetOperation().(*UpdateWorkerVersioningRulesRequest_ReplaceAssignmentRule); ok { + return x.ReplaceAssignmentRule + } + return nil +} + +func (x *UpdateWorkerVersioningRulesRequest) GetDeleteAssignmentRule() *UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule { + if x, ok := x.GetOperation().(*UpdateWorkerVersioningRulesRequest_DeleteAssignmentRule); ok { + return x.DeleteAssignmentRule + } + return nil +} + +func (x *UpdateWorkerVersioningRulesRequest) GetAddCompatibleRedirectRule() *UpdateWorkerVersioningRulesRequest_AddCompatibleBuildIdRedirectRule { + if x, ok := x.GetOperation().(*UpdateWorkerVersioningRulesRequest_AddCompatibleRedirectRule); ok { + return x.AddCompatibleRedirectRule + } + return nil +} + +func (x *UpdateWorkerVersioningRulesRequest) GetReplaceCompatibleRedirectRule() *UpdateWorkerVersioningRulesRequest_ReplaceCompatibleBuildIdRedirectRule { + if x, ok := x.GetOperation().(*UpdateWorkerVersioningRulesRequest_ReplaceCompatibleRedirectRule); ok { + return x.ReplaceCompatibleRedirectRule + } + return nil +} + +func (x *UpdateWorkerVersioningRulesRequest) GetDeleteCompatibleRedirectRule() *UpdateWorkerVersioningRulesRequest_DeleteCompatibleBuildIdRedirectRule { + if x, ok := x.GetOperation().(*UpdateWorkerVersioningRulesRequest_DeleteCompatibleRedirectRule); ok { + return x.DeleteCompatibleRedirectRule + } + return nil +} + +func (x *UpdateWorkerVersioningRulesRequest) GetCommitBuildId() *UpdateWorkerVersioningRulesRequest_CommitBuildId { + if x, ok := x.GetOperation().(*UpdateWorkerVersioningRulesRequest_CommitBuildId_); ok { + return x.CommitBuildId + } + return nil +} + +type isUpdateWorkerVersioningRulesRequest_Operation interface { + isUpdateWorkerVersioningRulesRequest_Operation() +} + +type UpdateWorkerVersioningRulesRequest_InsertAssignmentRule struct { + InsertAssignmentRule *UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule `protobuf:"bytes,4,opt,name=insert_assignment_rule,json=insertAssignmentRule,proto3,oneof"` +} + +type UpdateWorkerVersioningRulesRequest_ReplaceAssignmentRule struct { + ReplaceAssignmentRule *UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule `protobuf:"bytes,5,opt,name=replace_assignment_rule,json=replaceAssignmentRule,proto3,oneof"` +} + +type UpdateWorkerVersioningRulesRequest_DeleteAssignmentRule struct { + DeleteAssignmentRule *UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule `protobuf:"bytes,6,opt,name=delete_assignment_rule,json=deleteAssignmentRule,proto3,oneof"` +} + +type UpdateWorkerVersioningRulesRequest_AddCompatibleRedirectRule struct { + AddCompatibleRedirectRule *UpdateWorkerVersioningRulesRequest_AddCompatibleBuildIdRedirectRule `protobuf:"bytes,7,opt,name=add_compatible_redirect_rule,json=addCompatibleRedirectRule,proto3,oneof"` +} + +type UpdateWorkerVersioningRulesRequest_ReplaceCompatibleRedirectRule struct { + ReplaceCompatibleRedirectRule *UpdateWorkerVersioningRulesRequest_ReplaceCompatibleBuildIdRedirectRule `protobuf:"bytes,8,opt,name=replace_compatible_redirect_rule,json=replaceCompatibleRedirectRule,proto3,oneof"` +} + +type UpdateWorkerVersioningRulesRequest_DeleteCompatibleRedirectRule struct { + DeleteCompatibleRedirectRule *UpdateWorkerVersioningRulesRequest_DeleteCompatibleBuildIdRedirectRule `protobuf:"bytes,9,opt,name=delete_compatible_redirect_rule,json=deleteCompatibleRedirectRule,proto3,oneof"` +} + +type UpdateWorkerVersioningRulesRequest_CommitBuildId_ struct { + CommitBuildId *UpdateWorkerVersioningRulesRequest_CommitBuildId `protobuf:"bytes,10,opt,name=commit_build_id,json=commitBuildId,proto3,oneof"` +} + +func (*UpdateWorkerVersioningRulesRequest_InsertAssignmentRule) isUpdateWorkerVersioningRulesRequest_Operation() { +} + +func (*UpdateWorkerVersioningRulesRequest_ReplaceAssignmentRule) isUpdateWorkerVersioningRulesRequest_Operation() { +} + +func (*UpdateWorkerVersioningRulesRequest_DeleteAssignmentRule) isUpdateWorkerVersioningRulesRequest_Operation() { +} + +func (*UpdateWorkerVersioningRulesRequest_AddCompatibleRedirectRule) isUpdateWorkerVersioningRulesRequest_Operation() { +} + +func (*UpdateWorkerVersioningRulesRequest_ReplaceCompatibleRedirectRule) isUpdateWorkerVersioningRulesRequest_Operation() { +} + +func (*UpdateWorkerVersioningRulesRequest_DeleteCompatibleRedirectRule) isUpdateWorkerVersioningRulesRequest_Operation() { +} + +func (*UpdateWorkerVersioningRulesRequest_CommitBuildId_) isUpdateWorkerVersioningRulesRequest_Operation() { +} + +type UpdateWorkerVersioningRulesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + AssignmentRules []*v14.TimestampedBuildIdAssignmentRule `protobuf:"bytes,1,rep,name=assignment_rules,json=assignmentRules,proto3" json:"assignment_rules,omitempty"` + CompatibleRedirectRules []*v14.TimestampedCompatibleBuildIdRedirectRule `protobuf:"bytes,2,rep,name=compatible_redirect_rules,json=compatibleRedirectRules,proto3" json:"compatible_redirect_rules,omitempty"` + // This value can be passed back to UpdateWorkerVersioningRulesRequest to + // ensure that the rules were not modified between the two updates, which + // could lead to lost updates and other confusion. + ConflictToken []byte `protobuf:"bytes,3,opt,name=conflict_token,json=conflictToken,proto3" json:"conflict_token,omitempty"` +} + +func (x *UpdateWorkerVersioningRulesResponse) Reset() { + *x = UpdateWorkerVersioningRulesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[101] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkerVersioningRulesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkerVersioningRulesResponse) ProtoMessage() {} + +func (x *UpdateWorkerVersioningRulesResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[101] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkerVersioningRulesResponse.ProtoReflect.Descriptor instead. +func (*UpdateWorkerVersioningRulesResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{101} +} + +func (x *UpdateWorkerVersioningRulesResponse) GetAssignmentRules() []*v14.TimestampedBuildIdAssignmentRule { + if x != nil { + return x.AssignmentRules + } + return nil +} + +func (x *UpdateWorkerVersioningRulesResponse) GetCompatibleRedirectRules() []*v14.TimestampedCompatibleBuildIdRedirectRule { + if x != nil { + return x.CompatibleRedirectRules + } + return nil +} + +func (x *UpdateWorkerVersioningRulesResponse) GetConflictToken() []byte { + if x != nil { + return x.ConflictToken + } + return nil +} + +type GetWorkerVersioningRulesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + TaskQueue string `protobuf:"bytes,2,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` +} + +func (x *GetWorkerVersioningRulesRequest) Reset() { + *x = GetWorkerVersioningRulesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[102] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetWorkerVersioningRulesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetWorkerVersioningRulesRequest) ProtoMessage() {} + +func (x *GetWorkerVersioningRulesRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[102] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetWorkerVersioningRulesRequest.ProtoReflect.Descriptor instead. +func (*GetWorkerVersioningRulesRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{102} +} + +func (x *GetWorkerVersioningRulesRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *GetWorkerVersioningRulesRequest) GetTaskQueue() string { + if x != nil { + return x.TaskQueue + } + return "" +} + +type GetWorkerVersioningRulesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + AssignmentRules []*v14.TimestampedBuildIdAssignmentRule `protobuf:"bytes,1,rep,name=assignment_rules,json=assignmentRules,proto3" json:"assignment_rules,omitempty"` + CompatibleRedirectRules []*v14.TimestampedCompatibleBuildIdRedirectRule `protobuf:"bytes,2,rep,name=compatible_redirect_rules,json=compatibleRedirectRules,proto3" json:"compatible_redirect_rules,omitempty"` + // This value can be passed back to UpdateWorkerVersioningRulesRequest to + // ensure that the rules were not modified between this List and the Update, + // which could lead to lost updates and other confusion. + ConflictToken []byte `protobuf:"bytes,3,opt,name=conflict_token,json=conflictToken,proto3" json:"conflict_token,omitempty"` +} + +func (x *GetWorkerVersioningRulesResponse) Reset() { + *x = GetWorkerVersioningRulesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[103] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetWorkerVersioningRulesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetWorkerVersioningRulesResponse) ProtoMessage() {} + +func (x *GetWorkerVersioningRulesResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[103] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetWorkerVersioningRulesResponse.ProtoReflect.Descriptor instead. +func (*GetWorkerVersioningRulesResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{103} +} + +func (x *GetWorkerVersioningRulesResponse) GetAssignmentRules() []*v14.TimestampedBuildIdAssignmentRule { + if x != nil { + return x.AssignmentRules + } + return nil +} + +func (x *GetWorkerVersioningRulesResponse) GetCompatibleRedirectRules() []*v14.TimestampedCompatibleBuildIdRedirectRule { + if x != nil { + return x.CompatibleRedirectRules + } + return nil +} + +func (x *GetWorkerVersioningRulesResponse) GetConflictToken() []byte { + if x != nil { + return x.ConflictToken + } + return nil +} + +// Deprecated. Use `DescribeTaskQueue`. +type GetWorkerTaskReachabilityRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Build ids to retrieve reachability for. An empty string will be interpreted as an unversioned worker. + // The number of build ids that can be queried in a single API call is limited. + // Open source users can adjust this limit by setting the server's dynamic config value for + // `limit.reachabilityQueryBuildIds` with the caveat that this call can strain the visibility store. + BuildIds []string `protobuf:"bytes,2,rep,name=build_ids,json=buildIds,proto3" json:"build_ids,omitempty"` + // Task queues to retrieve reachability for. Leave this empty to query for all task queues associated with given + // build ids in the namespace. + // Must specify at least one task queue if querying for an unversioned worker. + // The number of task queues that the server will fetch reachability information for is limited. + // See the `GetWorkerTaskReachabilityResponse` documentation for more information. + TaskQueues []string `protobuf:"bytes,3,rep,name=task_queues,json=taskQueues,proto3" json:"task_queues,omitempty"` + // Type of reachability to query for. + // `TASK_REACHABILITY_NEW_WORKFLOWS` is always returned in the response. + // Use `TASK_REACHABILITY_EXISTING_WORKFLOWS` if your application needs to respond to queries on closed workflows. + // Otherwise, use `TASK_REACHABILITY_OPEN_WORKFLOWS`. Default is `TASK_REACHABILITY_EXISTING_WORKFLOWS` if left + // unspecified. + // See the TaskReachability docstring for information about each enum variant. + Reachability v11.TaskReachability `protobuf:"varint,4,opt,name=reachability,proto3,enum=temporal.api.enums.v1.TaskReachability" json:"reachability,omitempty"` +} + +func (x *GetWorkerTaskReachabilityRequest) Reset() { + *x = GetWorkerTaskReachabilityRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[104] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetWorkerTaskReachabilityRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetWorkerTaskReachabilityRequest) ProtoMessage() {} + +func (x *GetWorkerTaskReachabilityRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[104] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetWorkerTaskReachabilityRequest.ProtoReflect.Descriptor instead. +func (*GetWorkerTaskReachabilityRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{104} +} + +func (x *GetWorkerTaskReachabilityRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *GetWorkerTaskReachabilityRequest) GetBuildIds() []string { + if x != nil { + return x.BuildIds + } + return nil +} + +func (x *GetWorkerTaskReachabilityRequest) GetTaskQueues() []string { + if x != nil { + return x.TaskQueues + } + return nil +} + +func (x *GetWorkerTaskReachabilityRequest) GetReachability() v11.TaskReachability { + if x != nil { + return x.Reachability + } + return v11.TaskReachability(0) +} + +// Deprecated. Use `DescribeTaskQueue`. +type GetWorkerTaskReachabilityResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Task reachability, broken down by build id and then task queue. + // When requesting a large number of task queues or all task queues associated with the given build ids in a + // namespace, all task queues will be listed in the response but some of them may not contain reachability + // information due to a server enforced limit. When reaching the limit, task queues that reachability information + // could not be retrieved for will be marked with a single TASK_REACHABILITY_UNSPECIFIED entry. The caller may issue + // another call to get the reachability for those task queues. + // + // Open source users can adjust this limit by setting the server's dynamic config value for + // `limit.reachabilityTaskQueueScan` with the caveat that this call can strain the visibility store. + BuildIdReachability []*v14.BuildIdReachability `protobuf:"bytes,1,rep,name=build_id_reachability,json=buildIdReachability,proto3" json:"build_id_reachability,omitempty"` +} + +func (x *GetWorkerTaskReachabilityResponse) Reset() { + *x = GetWorkerTaskReachabilityResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[105] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetWorkerTaskReachabilityResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetWorkerTaskReachabilityResponse) ProtoMessage() {} + +func (x *GetWorkerTaskReachabilityResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[105] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetWorkerTaskReachabilityResponse.ProtoReflect.Descriptor instead. +func (*GetWorkerTaskReachabilityResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{105} +} + +func (x *GetWorkerTaskReachabilityResponse) GetBuildIdReachability() []*v14.BuildIdReachability { + if x != nil { + return x.BuildIdReachability + } + return nil +} + +// (-- api-linter: core::0134=disabled +// +// aip.dev/not-precedent: Update RPCs don't follow Google API format. --) +type UpdateWorkflowExecutionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace name of the target workflow + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The target workflow id and (optionally) a specific run thereof + // (-- api-linter: core::0203::optional=disabled + // + // aip.dev/not-precedent: false positive triggered by the word "optional" --) + WorkflowExecution *v13.WorkflowExecution `protobuf:"bytes,2,opt,name=workflow_execution,json=workflowExecution,proto3" json:"workflow_execution,omitempty"` + // If set, this call will error if the most recent (if no run id is set on + // `workflow_execution`), or specified (if it is) workflow execution is not + // part of the same execution chain as this id. + FirstExecutionRunId string `protobuf:"bytes,3,opt,name=first_execution_run_id,json=firstExecutionRunId,proto3" json:"first_execution_run_id,omitempty"` + // Describes when this request should return - basically whether the + // update is synchronous, asynchronous, or somewhere in between. + WaitPolicy *v115.WaitPolicy `protobuf:"bytes,4,opt,name=wait_policy,json=waitPolicy,proto3" json:"wait_policy,omitempty"` + // The request information that will be delivered all the way down to the + // workflow execution. + Request *v115.Request `protobuf:"bytes,5,opt,name=request,proto3" json:"request,omitempty"` +} + +func (x *UpdateWorkflowExecutionRequest) Reset() { + *x = UpdateWorkflowExecutionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[106] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkflowExecutionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkflowExecutionRequest) ProtoMessage() {} + +func (x *UpdateWorkflowExecutionRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[106] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkflowExecutionRequest.ProtoReflect.Descriptor instead. +func (*UpdateWorkflowExecutionRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{106} +} + +func (x *UpdateWorkflowExecutionRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *UpdateWorkflowExecutionRequest) GetWorkflowExecution() *v13.WorkflowExecution { + if x != nil { + return x.WorkflowExecution + } + return nil +} + +func (x *UpdateWorkflowExecutionRequest) GetFirstExecutionRunId() string { + if x != nil { + return x.FirstExecutionRunId + } + return "" +} + +func (x *UpdateWorkflowExecutionRequest) GetWaitPolicy() *v115.WaitPolicy { + if x != nil { + return x.WaitPolicy + } + return nil +} + +func (x *UpdateWorkflowExecutionRequest) GetRequest() *v115.Request { + if x != nil { + return x.Request + } + return nil +} + +type UpdateWorkflowExecutionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Enough information for subsequent poll calls if needed. Never null. + UpdateRef *v115.UpdateRef `protobuf:"bytes,1,opt,name=update_ref,json=updateRef,proto3" json:"update_ref,omitempty"` + // The outcome of the update if and only if the workflow execution update + // has completed. If this response is being returned before the update has + // completed then this field will not be set. + Outcome *v115.Outcome `protobuf:"bytes,2,opt,name=outcome,proto3" json:"outcome,omitempty"` + // The most advanced lifecycle stage that the Update is known to have + // reached, where lifecycle stages are ordered + // UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED < + // UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ADMITTED < + // UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ACCEPTED < + // UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED. + // UNSPECIFIED will be returned if and only if the server's maximum wait + // time was reached before the Update reached the stage specified in the + // request WaitPolicy, and before the context deadline expired; clients may + // may then retry the call as needed. + Stage v11.UpdateWorkflowExecutionLifecycleStage `protobuf:"varint,3,opt,name=stage,proto3,enum=temporal.api.enums.v1.UpdateWorkflowExecutionLifecycleStage" json:"stage,omitempty"` +} + +func (x *UpdateWorkflowExecutionResponse) Reset() { + *x = UpdateWorkflowExecutionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[107] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkflowExecutionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkflowExecutionResponse) ProtoMessage() {} + +func (x *UpdateWorkflowExecutionResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[107] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkflowExecutionResponse.ProtoReflect.Descriptor instead. +func (*UpdateWorkflowExecutionResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{107} +} + +func (x *UpdateWorkflowExecutionResponse) GetUpdateRef() *v115.UpdateRef { + if x != nil { + return x.UpdateRef + } + return nil +} + +func (x *UpdateWorkflowExecutionResponse) GetOutcome() *v115.Outcome { + if x != nil { + return x.Outcome + } + return nil +} + +func (x *UpdateWorkflowExecutionResponse) GetStage() v11.UpdateWorkflowExecutionLifecycleStage { + if x != nil { + return x.Stage + } + return v11.UpdateWorkflowExecutionLifecycleStage(0) +} + +type StartBatchOperationRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace that contains the batch operation + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Visibility query defines the the group of workflow to apply the batch operation + // This field and `executions` are mutually exclusive + VisibilityQuery string `protobuf:"bytes,2,opt,name=visibility_query,json=visibilityQuery,proto3" json:"visibility_query,omitempty"` + // Job ID defines the unique ID for the batch job + JobId string `protobuf:"bytes,3,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` + // Reason to perform the batch operation + Reason string `protobuf:"bytes,4,opt,name=reason,proto3" json:"reason,omitempty"` + // Executions to apply the batch operation + // This field and `visibility_query` are mutually exclusive + Executions []*v13.WorkflowExecution `protobuf:"bytes,5,rep,name=executions,proto3" json:"executions,omitempty"` + // Limit for the number of operations processed per second within this batch. + // Its purpose is to reduce the stress on the system caused by batch operations, which helps to prevent system + // overload and minimize potential delays in executing ongoing tasks for user workers. + // Note that when no explicit limit is provided, the server will operate according to its limit defined by the + // dynamic configuration key `worker.batcherRPS`. This also applies if the value in this field exceeds the + // server's configured limit. + MaxOperationsPerSecond float32 `protobuf:"fixed32,6,opt,name=max_operations_per_second,json=maxOperationsPerSecond,proto3" json:"max_operations_per_second,omitempty"` + // Operation input + // + // Types that are assignable to Operation: + // + // *StartBatchOperationRequest_TerminationOperation + // *StartBatchOperationRequest_SignalOperation + // *StartBatchOperationRequest_CancellationOperation + // *StartBatchOperationRequest_DeletionOperation + // *StartBatchOperationRequest_ResetOperation + Operation isStartBatchOperationRequest_Operation `protobuf_oneof:"operation"` +} + +func (x *StartBatchOperationRequest) Reset() { + *x = StartBatchOperationRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[108] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartBatchOperationRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartBatchOperationRequest) ProtoMessage() {} + +func (x *StartBatchOperationRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[108] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartBatchOperationRequest.ProtoReflect.Descriptor instead. +func (*StartBatchOperationRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{108} +} + +func (x *StartBatchOperationRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *StartBatchOperationRequest) GetVisibilityQuery() string { + if x != nil { + return x.VisibilityQuery + } + return "" +} + +func (x *StartBatchOperationRequest) GetJobId() string { + if x != nil { + return x.JobId + } + return "" +} + +func (x *StartBatchOperationRequest) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *StartBatchOperationRequest) GetExecutions() []*v13.WorkflowExecution { + if x != nil { + return x.Executions + } + return nil +} + +func (x *StartBatchOperationRequest) GetMaxOperationsPerSecond() float32 { + if x != nil { + return x.MaxOperationsPerSecond + } + return 0 +} + +func (m *StartBatchOperationRequest) GetOperation() isStartBatchOperationRequest_Operation { + if m != nil { + return m.Operation + } + return nil +} + +func (x *StartBatchOperationRequest) GetTerminationOperation() *v116.BatchOperationTermination { + if x, ok := x.GetOperation().(*StartBatchOperationRequest_TerminationOperation); ok { + return x.TerminationOperation + } + return nil +} + +func (x *StartBatchOperationRequest) GetSignalOperation() *v116.BatchOperationSignal { + if x, ok := x.GetOperation().(*StartBatchOperationRequest_SignalOperation); ok { + return x.SignalOperation + } + return nil +} + +func (x *StartBatchOperationRequest) GetCancellationOperation() *v116.BatchOperationCancellation { + if x, ok := x.GetOperation().(*StartBatchOperationRequest_CancellationOperation); ok { + return x.CancellationOperation + } + return nil +} + +func (x *StartBatchOperationRequest) GetDeletionOperation() *v116.BatchOperationDeletion { + if x, ok := x.GetOperation().(*StartBatchOperationRequest_DeletionOperation); ok { + return x.DeletionOperation + } + return nil +} + +func (x *StartBatchOperationRequest) GetResetOperation() *v116.BatchOperationReset { + if x, ok := x.GetOperation().(*StartBatchOperationRequest_ResetOperation); ok { + return x.ResetOperation + } + return nil +} + +type isStartBatchOperationRequest_Operation interface { + isStartBatchOperationRequest_Operation() +} + +type StartBatchOperationRequest_TerminationOperation struct { + TerminationOperation *v116.BatchOperationTermination `protobuf:"bytes,10,opt,name=termination_operation,json=terminationOperation,proto3,oneof"` +} + +type StartBatchOperationRequest_SignalOperation struct { + SignalOperation *v116.BatchOperationSignal `protobuf:"bytes,11,opt,name=signal_operation,json=signalOperation,proto3,oneof"` +} + +type StartBatchOperationRequest_CancellationOperation struct { + CancellationOperation *v116.BatchOperationCancellation `protobuf:"bytes,12,opt,name=cancellation_operation,json=cancellationOperation,proto3,oneof"` +} + +type StartBatchOperationRequest_DeletionOperation struct { + DeletionOperation *v116.BatchOperationDeletion `protobuf:"bytes,13,opt,name=deletion_operation,json=deletionOperation,proto3,oneof"` +} + +type StartBatchOperationRequest_ResetOperation struct { + ResetOperation *v116.BatchOperationReset `protobuf:"bytes,14,opt,name=reset_operation,json=resetOperation,proto3,oneof"` +} + +func (*StartBatchOperationRequest_TerminationOperation) isStartBatchOperationRequest_Operation() {} + +func (*StartBatchOperationRequest_SignalOperation) isStartBatchOperationRequest_Operation() {} + +func (*StartBatchOperationRequest_CancellationOperation) isStartBatchOperationRequest_Operation() {} + +func (*StartBatchOperationRequest_DeletionOperation) isStartBatchOperationRequest_Operation() {} + +func (*StartBatchOperationRequest_ResetOperation) isStartBatchOperationRequest_Operation() {} + +type StartBatchOperationResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *StartBatchOperationResponse) Reset() { + *x = StartBatchOperationResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[109] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StartBatchOperationResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartBatchOperationResponse) ProtoMessage() {} + +func (x *StartBatchOperationResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[109] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StartBatchOperationResponse.ProtoReflect.Descriptor instead. +func (*StartBatchOperationResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{109} +} + +type StopBatchOperationRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace that contains the batch operation + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Batch job id + JobId string `protobuf:"bytes,2,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` + // Reason to stop a batch operation + Reason string `protobuf:"bytes,3,opt,name=reason,proto3" json:"reason,omitempty"` + // Identity of the operator + Identity string `protobuf:"bytes,4,opt,name=identity,proto3" json:"identity,omitempty"` +} + +func (x *StopBatchOperationRequest) Reset() { + *x = StopBatchOperationRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[110] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StopBatchOperationRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StopBatchOperationRequest) ProtoMessage() {} + +func (x *StopBatchOperationRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[110] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StopBatchOperationRequest.ProtoReflect.Descriptor instead. +func (*StopBatchOperationRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{110} +} + +func (x *StopBatchOperationRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *StopBatchOperationRequest) GetJobId() string { + if x != nil { + return x.JobId + } + return "" +} + +func (x *StopBatchOperationRequest) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *StopBatchOperationRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +type StopBatchOperationResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *StopBatchOperationResponse) Reset() { + *x = StopBatchOperationResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[111] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StopBatchOperationResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StopBatchOperationResponse) ProtoMessage() {} + +func (x *StopBatchOperationResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[111] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StopBatchOperationResponse.ProtoReflect.Descriptor instead. +func (*StopBatchOperationResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{111} +} + +type DescribeBatchOperationRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace that contains the batch operation + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Batch job id + JobId string `protobuf:"bytes,2,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` +} + +func (x *DescribeBatchOperationRequest) Reset() { + *x = DescribeBatchOperationRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[112] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeBatchOperationRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeBatchOperationRequest) ProtoMessage() {} + +func (x *DescribeBatchOperationRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[112] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DescribeBatchOperationRequest.ProtoReflect.Descriptor instead. +func (*DescribeBatchOperationRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{112} +} + +func (x *DescribeBatchOperationRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *DescribeBatchOperationRequest) GetJobId() string { + if x != nil { + return x.JobId + } + return "" +} + +type DescribeBatchOperationResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Batch operation type + OperationType v11.BatchOperationType `protobuf:"varint,1,opt,name=operation_type,json=operationType,proto3,enum=temporal.api.enums.v1.BatchOperationType" json:"operation_type,omitempty"` + // Batch job ID + JobId string `protobuf:"bytes,2,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` + // Batch operation state + State v11.BatchOperationState `protobuf:"varint,3,opt,name=state,proto3,enum=temporal.api.enums.v1.BatchOperationState" json:"state,omitempty"` + // Batch operation start time + StartTime *timestamppb.Timestamp `protobuf:"bytes,4,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + // Batch operation close time + CloseTime *timestamppb.Timestamp `protobuf:"bytes,5,opt,name=close_time,json=closeTime,proto3" json:"close_time,omitempty"` + // Total operation count + TotalOperationCount int64 `protobuf:"varint,6,opt,name=total_operation_count,json=totalOperationCount,proto3" json:"total_operation_count,omitempty"` + // Complete operation count + CompleteOperationCount int64 `protobuf:"varint,7,opt,name=complete_operation_count,json=completeOperationCount,proto3" json:"complete_operation_count,omitempty"` + // Failure operation count + FailureOperationCount int64 `protobuf:"varint,8,opt,name=failure_operation_count,json=failureOperationCount,proto3" json:"failure_operation_count,omitempty"` + // Identity indicates the operator identity + Identity string `protobuf:"bytes,9,opt,name=identity,proto3" json:"identity,omitempty"` + // Reason indicates the reason to stop a operation + Reason string `protobuf:"bytes,10,opt,name=reason,proto3" json:"reason,omitempty"` +} + +func (x *DescribeBatchOperationResponse) Reset() { + *x = DescribeBatchOperationResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[113] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DescribeBatchOperationResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribeBatchOperationResponse) ProtoMessage() {} + +func (x *DescribeBatchOperationResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[113] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DescribeBatchOperationResponse.ProtoReflect.Descriptor instead. +func (*DescribeBatchOperationResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{113} +} + +func (x *DescribeBatchOperationResponse) GetOperationType() v11.BatchOperationType { + if x != nil { + return x.OperationType + } + return v11.BatchOperationType(0) +} + +func (x *DescribeBatchOperationResponse) GetJobId() string { + if x != nil { + return x.JobId + } + return "" +} + +func (x *DescribeBatchOperationResponse) GetState() v11.BatchOperationState { + if x != nil { + return x.State + } + return v11.BatchOperationState(0) +} + +func (x *DescribeBatchOperationResponse) GetStartTime() *timestamppb.Timestamp { + if x != nil { + return x.StartTime + } + return nil +} + +func (x *DescribeBatchOperationResponse) GetCloseTime() *timestamppb.Timestamp { + if x != nil { + return x.CloseTime + } + return nil +} + +func (x *DescribeBatchOperationResponse) GetTotalOperationCount() int64 { + if x != nil { + return x.TotalOperationCount + } + return 0 +} + +func (x *DescribeBatchOperationResponse) GetCompleteOperationCount() int64 { + if x != nil { + return x.CompleteOperationCount + } + return 0 +} + +func (x *DescribeBatchOperationResponse) GetFailureOperationCount() int64 { + if x != nil { + return x.FailureOperationCount + } + return 0 +} + +func (x *DescribeBatchOperationResponse) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *DescribeBatchOperationResponse) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +type ListBatchOperationsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Namespace that contains the batch operation + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // List page size + PageSize int32 `protobuf:"varint,2,opt,name=page_size,json=pageSize,proto3" json:"page_size,omitempty"` + // Next page token + NextPageToken []byte `protobuf:"bytes,3,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *ListBatchOperationsRequest) Reset() { + *x = ListBatchOperationsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[114] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListBatchOperationsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListBatchOperationsRequest) ProtoMessage() {} + +func (x *ListBatchOperationsRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[114] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListBatchOperationsRequest.ProtoReflect.Descriptor instead. +func (*ListBatchOperationsRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{114} +} + +func (x *ListBatchOperationsRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ListBatchOperationsRequest) GetPageSize() int32 { + if x != nil { + return x.PageSize + } + return 0 +} + +func (x *ListBatchOperationsRequest) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +type ListBatchOperationsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // BatchOperationInfo contains the basic info about batch operation + OperationInfo []*v116.BatchOperationInfo `protobuf:"bytes,1,rep,name=operation_info,json=operationInfo,proto3" json:"operation_info,omitempty"` + NextPageToken []byte `protobuf:"bytes,2,opt,name=next_page_token,json=nextPageToken,proto3" json:"next_page_token,omitempty"` +} + +func (x *ListBatchOperationsResponse) Reset() { + *x = ListBatchOperationsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[115] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListBatchOperationsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListBatchOperationsResponse) ProtoMessage() {} + +func (x *ListBatchOperationsResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[115] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListBatchOperationsResponse.ProtoReflect.Descriptor instead. +func (*ListBatchOperationsResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{115} +} + +func (x *ListBatchOperationsResponse) GetOperationInfo() []*v116.BatchOperationInfo { + if x != nil { + return x.OperationInfo + } + return nil +} + +func (x *ListBatchOperationsResponse) GetNextPageToken() []byte { + if x != nil { + return x.NextPageToken + } + return nil +} + +type PollWorkflowExecutionUpdateRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace of the workflow execution to which the update was + // originally issued. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The update reference returned in the initial + // UpdateWorkflowExecutionResponse + UpdateRef *v115.UpdateRef `protobuf:"bytes,2,opt,name=update_ref,json=updateRef,proto3" json:"update_ref,omitempty"` + // The identity of the worker/client who is polling this update outcome + Identity string `protobuf:"bytes,3,opt,name=identity,proto3" json:"identity,omitempty"` + // Describes when this poll request should return a response. + // Omit to request a non-blocking poll. + WaitPolicy *v115.WaitPolicy `protobuf:"bytes,4,opt,name=wait_policy,json=waitPolicy,proto3" json:"wait_policy,omitempty"` +} + +func (x *PollWorkflowExecutionUpdateRequest) Reset() { + *x = PollWorkflowExecutionUpdateRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[116] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PollWorkflowExecutionUpdateRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PollWorkflowExecutionUpdateRequest) ProtoMessage() {} + +func (x *PollWorkflowExecutionUpdateRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[116] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PollWorkflowExecutionUpdateRequest.ProtoReflect.Descriptor instead. +func (*PollWorkflowExecutionUpdateRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{116} +} + +func (x *PollWorkflowExecutionUpdateRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *PollWorkflowExecutionUpdateRequest) GetUpdateRef() *v115.UpdateRef { + if x != nil { + return x.UpdateRef + } + return nil +} + +func (x *PollWorkflowExecutionUpdateRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *PollWorkflowExecutionUpdateRequest) GetWaitPolicy() *v115.WaitPolicy { + if x != nil { + return x.WaitPolicy + } + return nil +} + +type PollWorkflowExecutionUpdateResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The outcome of the update if and only if the update has completed. If + // this response is being returned before the update has completed (e.g. due + // to the specification of a wait policy that only waits on + // UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ACCEPTED) then this field will + // not be set. + Outcome *v115.Outcome `protobuf:"bytes,1,opt,name=outcome,proto3" json:"outcome,omitempty"` + // The most advanced lifecycle stage that the Update is known to have + // reached, where lifecycle stages are ordered + // UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED < + // UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ADMITTED < + // UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ACCEPTED < + // UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED. + // UNSPECIFIED will be returned if and only if the server's maximum wait + // time was reached before the Update reached the stage specified in the + // request WaitPolicy, and before the context deadline expired; clients may + // may then retry the call as needed. + Stage v11.UpdateWorkflowExecutionLifecycleStage `protobuf:"varint,2,opt,name=stage,proto3,enum=temporal.api.enums.v1.UpdateWorkflowExecutionLifecycleStage" json:"stage,omitempty"` + // Sufficient information to address this update. + UpdateRef *v115.UpdateRef `protobuf:"bytes,3,opt,name=update_ref,json=updateRef,proto3" json:"update_ref,omitempty"` +} + +func (x *PollWorkflowExecutionUpdateResponse) Reset() { + *x = PollWorkflowExecutionUpdateResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[117] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PollWorkflowExecutionUpdateResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PollWorkflowExecutionUpdateResponse) ProtoMessage() {} + +func (x *PollWorkflowExecutionUpdateResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[117] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PollWorkflowExecutionUpdateResponse.ProtoReflect.Descriptor instead. +func (*PollWorkflowExecutionUpdateResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{117} +} + +func (x *PollWorkflowExecutionUpdateResponse) GetOutcome() *v115.Outcome { + if x != nil { + return x.Outcome + } + return nil +} + +func (x *PollWorkflowExecutionUpdateResponse) GetStage() v11.UpdateWorkflowExecutionLifecycleStage { + if x != nil { + return x.Stage + } + return v11.UpdateWorkflowExecutionLifecycleStage(0) +} + +func (x *PollWorkflowExecutionUpdateResponse) GetUpdateRef() *v115.UpdateRef { + if x != nil { + return x.UpdateRef + } + return nil +} + +type PollNexusTaskQueueRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The identity of the client who initiated this request. + Identity string `protobuf:"bytes,2,opt,name=identity,proto3" json:"identity,omitempty"` + TaskQueue *v14.TaskQueue `protobuf:"bytes,3,opt,name=task_queue,json=taskQueue,proto3" json:"task_queue,omitempty"` + // Information about this worker's build identifier and if it is choosing to use the versioning + // feature. See the `WorkerVersionCapabilities` docstring for more. + WorkerVersionCapabilities *v13.WorkerVersionCapabilities `protobuf:"bytes,4,opt,name=worker_version_capabilities,json=workerVersionCapabilities,proto3" json:"worker_version_capabilities,omitempty"` +} + +func (x *PollNexusTaskQueueRequest) Reset() { + *x = PollNexusTaskQueueRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[118] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PollNexusTaskQueueRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PollNexusTaskQueueRequest) ProtoMessage() {} + +func (x *PollNexusTaskQueueRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[118] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PollNexusTaskQueueRequest.ProtoReflect.Descriptor instead. +func (*PollNexusTaskQueueRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{118} +} + +func (x *PollNexusTaskQueueRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *PollNexusTaskQueueRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *PollNexusTaskQueueRequest) GetTaskQueue() *v14.TaskQueue { + if x != nil { + return x.TaskQueue + } + return nil +} + +func (x *PollNexusTaskQueueRequest) GetWorkerVersionCapabilities() *v13.WorkerVersionCapabilities { + if x != nil { + return x.WorkerVersionCapabilities + } + return nil +} + +type PollNexusTaskQueueResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // An opaque unique identifier for this task for correlating a completion request the embedded request. + TaskToken []byte `protobuf:"bytes,1,opt,name=task_token,json=taskToken,proto3" json:"task_token,omitempty"` + // Embedded request as translated from the incoming frontend request. + Request *v117.Request `protobuf:"bytes,2,opt,name=request,proto3" json:"request,omitempty"` +} + +func (x *PollNexusTaskQueueResponse) Reset() { + *x = PollNexusTaskQueueResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[119] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PollNexusTaskQueueResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PollNexusTaskQueueResponse) ProtoMessage() {} + +func (x *PollNexusTaskQueueResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[119] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PollNexusTaskQueueResponse.ProtoReflect.Descriptor instead. +func (*PollNexusTaskQueueResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{119} +} + +func (x *PollNexusTaskQueueResponse) GetTaskToken() []byte { + if x != nil { + return x.TaskToken + } + return nil +} + +func (x *PollNexusTaskQueueResponse) GetRequest() *v117.Request { + if x != nil { + return x.Request + } + return nil +} + +type RespondNexusTaskCompletedRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The identity of the client who initiated this request. + Identity string `protobuf:"bytes,2,opt,name=identity,proto3" json:"identity,omitempty"` + // A unique identifier for this task as received via a poll response. + TaskToken []byte `protobuf:"bytes,3,opt,name=task_token,json=taskToken,proto3" json:"task_token,omitempty"` + // Embedded response to be translated into a frontend response. + Response *v117.Response `protobuf:"bytes,4,opt,name=response,proto3" json:"response,omitempty"` +} + +func (x *RespondNexusTaskCompletedRequest) Reset() { + *x = RespondNexusTaskCompletedRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[120] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondNexusTaskCompletedRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondNexusTaskCompletedRequest) ProtoMessage() {} + +func (x *RespondNexusTaskCompletedRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[120] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondNexusTaskCompletedRequest.ProtoReflect.Descriptor instead. +func (*RespondNexusTaskCompletedRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{120} +} + +func (x *RespondNexusTaskCompletedRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RespondNexusTaskCompletedRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *RespondNexusTaskCompletedRequest) GetTaskToken() []byte { + if x != nil { + return x.TaskToken + } + return nil +} + +func (x *RespondNexusTaskCompletedRequest) GetResponse() *v117.Response { + if x != nil { + return x.Response + } + return nil +} + +type RespondNexusTaskCompletedResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RespondNexusTaskCompletedResponse) Reset() { + *x = RespondNexusTaskCompletedResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[121] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondNexusTaskCompletedResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondNexusTaskCompletedResponse) ProtoMessage() {} + +func (x *RespondNexusTaskCompletedResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[121] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondNexusTaskCompletedResponse.ProtoReflect.Descriptor instead. +func (*RespondNexusTaskCompletedResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{121} +} + +type RespondNexusTaskFailedRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // The identity of the client who initiated this request. + Identity string `protobuf:"bytes,2,opt,name=identity,proto3" json:"identity,omitempty"` + // A unique identifier for this task. + TaskToken []byte `protobuf:"bytes,3,opt,name=task_token,json=taskToken,proto3" json:"task_token,omitempty"` + // The error the handler failed with. + Error *v117.HandlerError `protobuf:"bytes,4,opt,name=error,proto3" json:"error,omitempty"` +} + +func (x *RespondNexusTaskFailedRequest) Reset() { + *x = RespondNexusTaskFailedRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[122] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondNexusTaskFailedRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondNexusTaskFailedRequest) ProtoMessage() {} + +func (x *RespondNexusTaskFailedRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[122] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondNexusTaskFailedRequest.ProtoReflect.Descriptor instead. +func (*RespondNexusTaskFailedRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{122} +} + +func (x *RespondNexusTaskFailedRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RespondNexusTaskFailedRequest) GetIdentity() string { + if x != nil { + return x.Identity + } + return "" +} + +func (x *RespondNexusTaskFailedRequest) GetTaskToken() []byte { + if x != nil { + return x.TaskToken + } + return nil +} + +func (x *RespondNexusTaskFailedRequest) GetError() *v117.HandlerError { + if x != nil { + return x.Error + } + return nil +} + +type RespondNexusTaskFailedResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *RespondNexusTaskFailedResponse) Reset() { + *x = RespondNexusTaskFailedResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[123] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RespondNexusTaskFailedResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RespondNexusTaskFailedResponse) ProtoMessage() {} + +func (x *RespondNexusTaskFailedResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[123] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RespondNexusTaskFailedResponse.ProtoReflect.Descriptor instead. +func (*RespondNexusTaskFailedResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{123} +} + +type ExecuteMultiOperationRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + // List of operations to execute within a single workflow. + // + // Preconditions: + // - The list of operations must not be empty. + // - The workflow ids must match across operations. + // - The only valid list of operations at this time is [StartWorkflow, UpdateWorkflow], in this order. + // + // Note that additional operation-specific restrictions have to be considered. + Operations []*ExecuteMultiOperationRequest_Operation `protobuf:"bytes,2,rep,name=operations,proto3" json:"operations,omitempty"` +} + +func (x *ExecuteMultiOperationRequest) Reset() { + *x = ExecuteMultiOperationRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[124] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ExecuteMultiOperationRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecuteMultiOperationRequest) ProtoMessage() {} + +func (x *ExecuteMultiOperationRequest) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[124] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ExecuteMultiOperationRequest.ProtoReflect.Descriptor instead. +func (*ExecuteMultiOperationRequest) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{124} +} + +func (x *ExecuteMultiOperationRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ExecuteMultiOperationRequest) GetOperations() []*ExecuteMultiOperationRequest_Operation { + if x != nil { + return x.Operations + } + return nil +} + +type ExecuteMultiOperationResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Responses []*ExecuteMultiOperationResponse_Response `protobuf:"bytes,1,rep,name=responses,proto3" json:"responses,omitempty"` +} + +func (x *ExecuteMultiOperationResponse) Reset() { + *x = ExecuteMultiOperationResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[125] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ExecuteMultiOperationResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecuteMultiOperationResponse) ProtoMessage() {} + +func (x *ExecuteMultiOperationResponse) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[125] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ExecuteMultiOperationResponse.ProtoReflect.Descriptor instead. +func (*ExecuteMultiOperationResponse) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{125} +} + +func (x *ExecuteMultiOperationResponse) GetResponses() []*ExecuteMultiOperationResponse_Response { + if x != nil { + return x.Responses + } + return nil +} + +type CountWorkflowExecutionsResponse_AggregationGroup struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + GroupValues []*v13.Payload `protobuf:"bytes,1,rep,name=group_values,json=groupValues,proto3" json:"group_values,omitempty"` + Count int64 `protobuf:"varint,2,opt,name=count,proto3" json:"count,omitempty"` +} + +func (x *CountWorkflowExecutionsResponse_AggregationGroup) Reset() { + *x = CountWorkflowExecutionsResponse_AggregationGroup{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[129] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CountWorkflowExecutionsResponse_AggregationGroup) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CountWorkflowExecutionsResponse_AggregationGroup) ProtoMessage() {} + +func (x *CountWorkflowExecutionsResponse_AggregationGroup) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[129] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CountWorkflowExecutionsResponse_AggregationGroup.ProtoReflect.Descriptor instead. +func (*CountWorkflowExecutionsResponse_AggregationGroup) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{63, 0} +} + +func (x *CountWorkflowExecutionsResponse_AggregationGroup) GetGroupValues() []*v13.Payload { + if x != nil { + return x.GroupValues + } + return nil +} + +func (x *CountWorkflowExecutionsResponse_AggregationGroup) GetCount() int64 { + if x != nil { + return x.Count + } + return 0 +} + +// System capability details. +type GetSystemInfoResponse_Capabilities struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // True if signal and query headers are supported. + SignalAndQueryHeader bool `protobuf:"varint,1,opt,name=signal_and_query_header,json=signalAndQueryHeader,proto3" json:"signal_and_query_header,omitempty"` + // True if internal errors are differentiated from other types of errors for purposes of + // retrying non-internal errors. + // + // When unset/false, clients retry all failures. When true, clients should only retry + // non-internal errors. + InternalErrorDifferentiation bool `protobuf:"varint,2,opt,name=internal_error_differentiation,json=internalErrorDifferentiation,proto3" json:"internal_error_differentiation,omitempty"` + // True if RespondActivityTaskFailed API supports including heartbeat details + ActivityFailureIncludeHeartbeat bool `protobuf:"varint,3,opt,name=activity_failure_include_heartbeat,json=activityFailureIncludeHeartbeat,proto3" json:"activity_failure_include_heartbeat,omitempty"` + // Supports scheduled workflow features. + SupportsSchedules bool `protobuf:"varint,4,opt,name=supports_schedules,json=supportsSchedules,proto3" json:"supports_schedules,omitempty"` + // True if server uses protos that include temporal.api.failure.v1.Failure.encoded_attributes + EncodedFailureAttributes bool `protobuf:"varint,5,opt,name=encoded_failure_attributes,json=encodedFailureAttributes,proto3" json:"encoded_failure_attributes,omitempty"` + // True if server supports dispatching Workflow and Activity tasks based on a worker's build_id + // (see: + // https://github.com/temporalio/proposals/blob/a123af3b559f43db16ea6dd31870bfb754c4dc5e/versioning/worker-versions.md) + BuildIdBasedVersioning bool `protobuf:"varint,6,opt,name=build_id_based_versioning,json=buildIdBasedVersioning,proto3" json:"build_id_based_versioning,omitempty"` + // True if server supports upserting workflow memo + UpsertMemo bool `protobuf:"varint,7,opt,name=upsert_memo,json=upsertMemo,proto3" json:"upsert_memo,omitempty"` + // True if server supports eager workflow task dispatching for the StartWorkflowExecution API + EagerWorkflowStart bool `protobuf:"varint,8,opt,name=eager_workflow_start,json=eagerWorkflowStart,proto3" json:"eager_workflow_start,omitempty"` + // True if the server knows about the sdk metadata field on WFT completions and will record + // it in history + SdkMetadata bool `protobuf:"varint,9,opt,name=sdk_metadata,json=sdkMetadata,proto3" json:"sdk_metadata,omitempty"` + // True if the server supports count group by execution status + // (-- api-linter: core::0140::prepositions=disabled --) + CountGroupByExecutionStatus bool `protobuf:"varint,10,opt,name=count_group_by_execution_status,json=countGroupByExecutionStatus,proto3" json:"count_group_by_execution_status,omitempty"` + // True if the server supports Nexus operations. + // This flag is dependent both on server version and for Nexus to be enabled via server configuration. + Nexus bool `protobuf:"varint,11,opt,name=nexus,proto3" json:"nexus,omitempty"` +} + +func (x *GetSystemInfoResponse_Capabilities) Reset() { + *x = GetSystemInfoResponse_Capabilities{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[133] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetSystemInfoResponse_Capabilities) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetSystemInfoResponse_Capabilities) ProtoMessage() {} + +func (x *GetSystemInfoResponse_Capabilities) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[133] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetSystemInfoResponse_Capabilities.ProtoReflect.Descriptor instead. +func (*GetSystemInfoResponse_Capabilities) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{79, 0} +} + +func (x *GetSystemInfoResponse_Capabilities) GetSignalAndQueryHeader() bool { + if x != nil { + return x.SignalAndQueryHeader + } + return false +} + +func (x *GetSystemInfoResponse_Capabilities) GetInternalErrorDifferentiation() bool { + if x != nil { + return x.InternalErrorDifferentiation + } + return false +} + +func (x *GetSystemInfoResponse_Capabilities) GetActivityFailureIncludeHeartbeat() bool { + if x != nil { + return x.ActivityFailureIncludeHeartbeat + } + return false +} + +func (x *GetSystemInfoResponse_Capabilities) GetSupportsSchedules() bool { + if x != nil { + return x.SupportsSchedules + } + return false +} + +func (x *GetSystemInfoResponse_Capabilities) GetEncodedFailureAttributes() bool { + if x != nil { + return x.EncodedFailureAttributes + } + return false +} + +func (x *GetSystemInfoResponse_Capabilities) GetBuildIdBasedVersioning() bool { + if x != nil { + return x.BuildIdBasedVersioning + } + return false +} + +func (x *GetSystemInfoResponse_Capabilities) GetUpsertMemo() bool { + if x != nil { + return x.UpsertMemo + } + return false +} + +func (x *GetSystemInfoResponse_Capabilities) GetEagerWorkflowStart() bool { + if x != nil { + return x.EagerWorkflowStart + } + return false +} + +func (x *GetSystemInfoResponse_Capabilities) GetSdkMetadata() bool { + if x != nil { + return x.SdkMetadata + } + return false +} + +func (x *GetSystemInfoResponse_Capabilities) GetCountGroupByExecutionStatus() bool { + if x != nil { + return x.CountGroupByExecutionStatus + } + return false +} + +func (x *GetSystemInfoResponse_Capabilities) GetNexus() bool { + if x != nil { + return x.Nexus + } + return false +} + +type UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // A new id to be added to an existing compatible set. + NewBuildId string `protobuf:"bytes,1,opt,name=new_build_id,json=newBuildId,proto3" json:"new_build_id,omitempty"` + // A build id which must already exist in the version sets known by the task queue. The new + // id will be stored in the set containing this id, marking it as compatible with + // the versions within. + ExistingCompatibleBuildId string `protobuf:"bytes,2,opt,name=existing_compatible_build_id,json=existingCompatibleBuildId,proto3" json:"existing_compatible_build_id,omitempty"` + // When set, establishes the compatible set being targeted as the overall default for the + // queue. If a different set was the current default, the targeted set will replace it as + // the new default. + MakeSetDefault bool `protobuf:"varint,3,opt,name=make_set_default,json=makeSetDefault,proto3" json:"make_set_default,omitempty"` +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion) Reset() { + *x = UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[134] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion) ProtoMessage() {} + +func (x *UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[134] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion.ProtoReflect.Descriptor instead. +func (*UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{96, 0} +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion) GetNewBuildId() string { + if x != nil { + return x.NewBuildId + } + return "" +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion) GetExistingCompatibleBuildId() string { + if x != nil { + return x.ExistingCompatibleBuildId + } + return "" +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion) GetMakeSetDefault() bool { + if x != nil { + return x.MakeSetDefault + } + return false +} + +type UpdateWorkerBuildIdCompatibilityRequest_MergeSets struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // A build ID in the set whose default will become the merged set default + PrimarySetBuildId string `protobuf:"bytes,1,opt,name=primary_set_build_id,json=primarySetBuildId,proto3" json:"primary_set_build_id,omitempty"` + // A build ID in the set which will be merged into the primary set + SecondarySetBuildId string `protobuf:"bytes,2,opt,name=secondary_set_build_id,json=secondarySetBuildId,proto3" json:"secondary_set_build_id,omitempty"` +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest_MergeSets) Reset() { + *x = UpdateWorkerBuildIdCompatibilityRequest_MergeSets{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[135] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest_MergeSets) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkerBuildIdCompatibilityRequest_MergeSets) ProtoMessage() {} + +func (x *UpdateWorkerBuildIdCompatibilityRequest_MergeSets) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[135] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkerBuildIdCompatibilityRequest_MergeSets.ProtoReflect.Descriptor instead. +func (*UpdateWorkerBuildIdCompatibilityRequest_MergeSets) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{96, 1} +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest_MergeSets) GetPrimarySetBuildId() string { + if x != nil { + return x.PrimarySetBuildId + } + return "" +} + +func (x *UpdateWorkerBuildIdCompatibilityRequest_MergeSets) GetSecondarySetBuildId() string { + if x != nil { + return x.SecondarySetBuildId + } + return "" +} + +// Inserts the rule to the list of assignment rules for this Task Queue. +// The rules are evaluated in order, starting from index 0. The first +// applicable rule will be applied and the rest will be ignored. +type UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Use this option to insert the rule in a particular index. By + // default, the new rule is inserted at the beginning of the list + // (index 0). If the given index is too larger the rule will be + // inserted at the end of the list. + RuleIndex int32 `protobuf:"varint,1,opt,name=rule_index,json=ruleIndex,proto3" json:"rule_index,omitempty"` + Rule *v14.BuildIdAssignmentRule `protobuf:"bytes,2,opt,name=rule,proto3" json:"rule,omitempty"` +} + +func (x *UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule) Reset() { + *x = UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[136] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule) ProtoMessage() {} + +func (x *UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[136] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule.ProtoReflect.Descriptor instead. +func (*UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{100, 0} +} + +func (x *UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule) GetRuleIndex() int32 { + if x != nil { + return x.RuleIndex + } + return 0 +} + +func (x *UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule) GetRule() *v14.BuildIdAssignmentRule { + if x != nil { + return x.Rule + } + return nil +} + +// Replaces the assignment rule at a given index. +type UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + RuleIndex int32 `protobuf:"varint,1,opt,name=rule_index,json=ruleIndex,proto3" json:"rule_index,omitempty"` + Rule *v14.BuildIdAssignmentRule `protobuf:"bytes,2,opt,name=rule,proto3" json:"rule,omitempty"` + // By default presence of one unconditional rule is enforced, otherwise + // the replace operation will be rejected. Set `force` to true to + // bypass this validation. An unconditional assignment rule: + // - Has no hint filter + // - Has no ramp + Force bool `protobuf:"varint,3,opt,name=force,proto3" json:"force,omitempty"` +} + +func (x *UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule) Reset() { + *x = UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[137] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule) ProtoMessage() {} + +func (x *UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[137] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule.ProtoReflect.Descriptor instead. +func (*UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{100, 1} +} + +func (x *UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule) GetRuleIndex() int32 { + if x != nil { + return x.RuleIndex + } + return 0 +} + +func (x *UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule) GetRule() *v14.BuildIdAssignmentRule { + if x != nil { + return x.Rule + } + return nil +} + +func (x *UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule) GetForce() bool { + if x != nil { + return x.Force + } + return false +} + +type UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + RuleIndex int32 `protobuf:"varint,1,opt,name=rule_index,json=ruleIndex,proto3" json:"rule_index,omitempty"` + // By default presence of one unconditional rule is enforced, otherwise + // the delete operation will be rejected. Set `force` to true to + // bypass this validation. An unconditional assignment rule: + // - Has no hint filter + // - Has no ramp + Force bool `protobuf:"varint,2,opt,name=force,proto3" json:"force,omitempty"` +} + +func (x *UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule) Reset() { + *x = UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[138] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule) ProtoMessage() {} + +func (x *UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[138] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule.ProtoReflect.Descriptor instead. +func (*UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{100, 2} +} + +func (x *UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule) GetRuleIndex() int32 { + if x != nil { + return x.RuleIndex + } + return 0 +} + +func (x *UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule) GetForce() bool { + if x != nil { + return x.Force + } + return false +} + +// Adds the rule to the list of redirect rules for this Task Queue. There +// can be at most one redirect rule for each distinct Source Build ID. +type UpdateWorkerVersioningRulesRequest_AddCompatibleBuildIdRedirectRule struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Rule *v14.CompatibleBuildIdRedirectRule `protobuf:"bytes,1,opt,name=rule,proto3" json:"rule,omitempty"` +} + +func (x *UpdateWorkerVersioningRulesRequest_AddCompatibleBuildIdRedirectRule) Reset() { + *x = UpdateWorkerVersioningRulesRequest_AddCompatibleBuildIdRedirectRule{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[139] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkerVersioningRulesRequest_AddCompatibleBuildIdRedirectRule) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkerVersioningRulesRequest_AddCompatibleBuildIdRedirectRule) ProtoMessage() {} + +func (x *UpdateWorkerVersioningRulesRequest_AddCompatibleBuildIdRedirectRule) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[139] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkerVersioningRulesRequest_AddCompatibleBuildIdRedirectRule.ProtoReflect.Descriptor instead. +func (*UpdateWorkerVersioningRulesRequest_AddCompatibleBuildIdRedirectRule) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{100, 3} +} + +func (x *UpdateWorkerVersioningRulesRequest_AddCompatibleBuildIdRedirectRule) GetRule() *v14.CompatibleBuildIdRedirectRule { + if x != nil { + return x.Rule + } + return nil +} + +// Replaces the routing rule with the given source Build ID. +type UpdateWorkerVersioningRulesRequest_ReplaceCompatibleBuildIdRedirectRule struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Rule *v14.CompatibleBuildIdRedirectRule `protobuf:"bytes,1,opt,name=rule,proto3" json:"rule,omitempty"` +} + +func (x *UpdateWorkerVersioningRulesRequest_ReplaceCompatibleBuildIdRedirectRule) Reset() { + *x = UpdateWorkerVersioningRulesRequest_ReplaceCompatibleBuildIdRedirectRule{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[140] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkerVersioningRulesRequest_ReplaceCompatibleBuildIdRedirectRule) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkerVersioningRulesRequest_ReplaceCompatibleBuildIdRedirectRule) ProtoMessage() {} + +func (x *UpdateWorkerVersioningRulesRequest_ReplaceCompatibleBuildIdRedirectRule) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[140] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkerVersioningRulesRequest_ReplaceCompatibleBuildIdRedirectRule.ProtoReflect.Descriptor instead. +func (*UpdateWorkerVersioningRulesRequest_ReplaceCompatibleBuildIdRedirectRule) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{100, 4} +} + +func (x *UpdateWorkerVersioningRulesRequest_ReplaceCompatibleBuildIdRedirectRule) GetRule() *v14.CompatibleBuildIdRedirectRule { + if x != nil { + return x.Rule + } + return nil +} + +type UpdateWorkerVersioningRulesRequest_DeleteCompatibleBuildIdRedirectRule struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SourceBuildId string `protobuf:"bytes,1,opt,name=source_build_id,json=sourceBuildId,proto3" json:"source_build_id,omitempty"` +} + +func (x *UpdateWorkerVersioningRulesRequest_DeleteCompatibleBuildIdRedirectRule) Reset() { + *x = UpdateWorkerVersioningRulesRequest_DeleteCompatibleBuildIdRedirectRule{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[141] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkerVersioningRulesRequest_DeleteCompatibleBuildIdRedirectRule) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkerVersioningRulesRequest_DeleteCompatibleBuildIdRedirectRule) ProtoMessage() {} + +func (x *UpdateWorkerVersioningRulesRequest_DeleteCompatibleBuildIdRedirectRule) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[141] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkerVersioningRulesRequest_DeleteCompatibleBuildIdRedirectRule.ProtoReflect.Descriptor instead. +func (*UpdateWorkerVersioningRulesRequest_DeleteCompatibleBuildIdRedirectRule) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{100, 5} +} + +func (x *UpdateWorkerVersioningRulesRequest_DeleteCompatibleBuildIdRedirectRule) GetSourceBuildId() string { + if x != nil { + return x.SourceBuildId + } + return "" +} + +// This command is intended to be used to complete the rollout of a Build +// ID and cleanup unnecessary rules possibly created during a gradual +// rollout. Specifically, this command will make the following changes +// atomically: +// 1. Adds an assignment rule (with full ramp) for the target Build ID at +// the end of the list. +// 2. Removes all previously added assignment rules to the given target +// Build ID (if any). +// 3. Removes any fully-ramped assignment rule for other Build IDs. +type UpdateWorkerVersioningRulesRequest_CommitBuildId struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TargetBuildId string `protobuf:"bytes,1,opt,name=target_build_id,json=targetBuildId,proto3" json:"target_build_id,omitempty"` + // To prevent committing invalid Build IDs, we reject the request if no + // pollers has been seen recently for this Build ID. Use the `force` + // option to disable this validation. + Force bool `protobuf:"varint,2,opt,name=force,proto3" json:"force,omitempty"` +} + +func (x *UpdateWorkerVersioningRulesRequest_CommitBuildId) Reset() { + *x = UpdateWorkerVersioningRulesRequest_CommitBuildId{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[142] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateWorkerVersioningRulesRequest_CommitBuildId) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateWorkerVersioningRulesRequest_CommitBuildId) ProtoMessage() {} + +func (x *UpdateWorkerVersioningRulesRequest_CommitBuildId) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[142] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateWorkerVersioningRulesRequest_CommitBuildId.ProtoReflect.Descriptor instead. +func (*UpdateWorkerVersioningRulesRequest_CommitBuildId) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{100, 6} +} + +func (x *UpdateWorkerVersioningRulesRequest_CommitBuildId) GetTargetBuildId() string { + if x != nil { + return x.TargetBuildId + } + return "" +} + +func (x *UpdateWorkerVersioningRulesRequest_CommitBuildId) GetForce() bool { + if x != nil { + return x.Force + } + return false +} + +type ExecuteMultiOperationRequest_Operation struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Operation: + // + // *ExecuteMultiOperationRequest_Operation_StartWorkflow + // *ExecuteMultiOperationRequest_Operation_UpdateWorkflow + Operation isExecuteMultiOperationRequest_Operation_Operation `protobuf_oneof:"operation"` +} + +func (x *ExecuteMultiOperationRequest_Operation) Reset() { + *x = ExecuteMultiOperationRequest_Operation{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[143] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ExecuteMultiOperationRequest_Operation) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecuteMultiOperationRequest_Operation) ProtoMessage() {} + +func (x *ExecuteMultiOperationRequest_Operation) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[143] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ExecuteMultiOperationRequest_Operation.ProtoReflect.Descriptor instead. +func (*ExecuteMultiOperationRequest_Operation) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{124, 0} +} + +func (m *ExecuteMultiOperationRequest_Operation) GetOperation() isExecuteMultiOperationRequest_Operation_Operation { + if m != nil { + return m.Operation + } + return nil +} + +func (x *ExecuteMultiOperationRequest_Operation) GetStartWorkflow() *StartWorkflowExecutionRequest { + if x, ok := x.GetOperation().(*ExecuteMultiOperationRequest_Operation_StartWorkflow); ok { + return x.StartWorkflow + } + return nil +} + +func (x *ExecuteMultiOperationRequest_Operation) GetUpdateWorkflow() *UpdateWorkflowExecutionRequest { + if x, ok := x.GetOperation().(*ExecuteMultiOperationRequest_Operation_UpdateWorkflow); ok { + return x.UpdateWorkflow + } + return nil +} + +type isExecuteMultiOperationRequest_Operation_Operation interface { + isExecuteMultiOperationRequest_Operation_Operation() +} + +type ExecuteMultiOperationRequest_Operation_StartWorkflow struct { + // Additional restrictions: + // - setting `cron_schedule` is invalid + // - setting `request_eager_execution` is invalid + // - setting `workflow_start_delay` is invalid + StartWorkflow *StartWorkflowExecutionRequest `protobuf:"bytes,1,opt,name=start_workflow,json=startWorkflow,proto3,oneof"` +} + +type ExecuteMultiOperationRequest_Operation_UpdateWorkflow struct { + // Additional restrictions: + // - setting `first_execution_run_id` is invalid + // - setting `workflow_execution.run_id` is invalid + UpdateWorkflow *UpdateWorkflowExecutionRequest `protobuf:"bytes,2,opt,name=update_workflow,json=updateWorkflow,proto3,oneof"` +} + +func (*ExecuteMultiOperationRequest_Operation_StartWorkflow) isExecuteMultiOperationRequest_Operation_Operation() { +} + +func (*ExecuteMultiOperationRequest_Operation_UpdateWorkflow) isExecuteMultiOperationRequest_Operation_Operation() { +} + +type ExecuteMultiOperationResponse_Response struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Response: + // + // *ExecuteMultiOperationResponse_Response_StartWorkflow + // *ExecuteMultiOperationResponse_Response_UpdateWorkflow + Response isExecuteMultiOperationResponse_Response_Response `protobuf_oneof:"response"` +} + +func (x *ExecuteMultiOperationResponse_Response) Reset() { + *x = ExecuteMultiOperationResponse_Response{} + if protoimpl.UnsafeEnabled { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[144] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ExecuteMultiOperationResponse_Response) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecuteMultiOperationResponse_Response) ProtoMessage() {} + +func (x *ExecuteMultiOperationResponse_Response) ProtoReflect() protoreflect.Message { + mi := &file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[144] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ExecuteMultiOperationResponse_Response.ProtoReflect.Descriptor instead. +func (*ExecuteMultiOperationResponse_Response) Descriptor() ([]byte, []int) { + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP(), []int{125, 0} +} + +func (m *ExecuteMultiOperationResponse_Response) GetResponse() isExecuteMultiOperationResponse_Response_Response { + if m != nil { + return m.Response + } + return nil +} + +func (x *ExecuteMultiOperationResponse_Response) GetStartWorkflow() *StartWorkflowExecutionResponse { + if x, ok := x.GetResponse().(*ExecuteMultiOperationResponse_Response_StartWorkflow); ok { + return x.StartWorkflow + } + return nil +} + +func (x *ExecuteMultiOperationResponse_Response) GetUpdateWorkflow() *UpdateWorkflowExecutionResponse { + if x, ok := x.GetResponse().(*ExecuteMultiOperationResponse_Response_UpdateWorkflow); ok { + return x.UpdateWorkflow + } + return nil +} + +type isExecuteMultiOperationResponse_Response_Response interface { + isExecuteMultiOperationResponse_Response_Response() +} + +type ExecuteMultiOperationResponse_Response_StartWorkflow struct { + StartWorkflow *StartWorkflowExecutionResponse `protobuf:"bytes,1,opt,name=start_workflow,json=startWorkflow,proto3,oneof"` +} + +type ExecuteMultiOperationResponse_Response_UpdateWorkflow struct { + UpdateWorkflow *UpdateWorkflowExecutionResponse `protobuf:"bytes,2,opt,name=update_workflow,json=updateWorkflow,proto3,oneof"` +} + +func (*ExecuteMultiOperationResponse_Response_StartWorkflow) isExecuteMultiOperationResponse_Response_Response() { +} + +func (*ExecuteMultiOperationResponse_Response_UpdateWorkflow) isExecuteMultiOperationResponse_Response_Response() { +} + +var File_temporal_api_workflowservice_v1_request_response_proto protoreflect.FileDescriptor + +var file_temporal_api_workflowservice_v1_request_response_proto_rawDesc = []byte{ + 0x0a, 0x36, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, 0x76, 0x31, 0x2f, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x1f, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x1a, 0x2b, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, + 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, + 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x1a, 0x25, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, + 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x28, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x66, 0x61, 0x69, + 0x6c, 0x65, 0x64, 0x5f, 0x63, 0x61, 0x75, 0x73, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x22, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, + 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x1a, 0x21, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x1a, 0x21, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, + 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x72, 0x65, 0x73, 0x65, 0x74, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x26, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, + 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, + 0x65, 0x75, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x22, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x75, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x76, + 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x68, 0x69, 0x73, 0x74, + 0x6f, 0x72, 0x79, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x26, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2f, 0x76, + 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x25, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x66, 0x61, 0x69, 0x6c, + 0x75, 0x72, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, + 0x2f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x26, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2f, 0x76, 0x31, 0x2f, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x23, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x29, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, + 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x26, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x73, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, + 0x70, 0x69, 0x2f, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2f, + 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x25, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x23, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x30, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x73, 0x64, 0x6b, 0x2f, 0x76, 0x31, 0x2f, 0x74, 0x61, 0x73, 0x6b, + 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x73, 0x64, 0x6b, 0x2f, 0x76, 0x31, 0x2f, 0x75, 0x73, 0x65, 0x72, 0x5f, + 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x23, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x6e, 0x65, 0x78, 0x75, 0x73, + 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x1a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xb5, 0x07, + 0x0a, 0x18, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x24, 0x0a, 0x0b, 0x64, 0x65, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, + 0x0b, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x5f, 0x65, 0x6d, 0x61, 0x69, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0a, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x45, 0x6d, 0x61, 0x69, 0x6c, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x6c, 0x0a, 0x23, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, + 0x65, 0x72, 0x69, 0x6f, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, + 0x72, 0x69, 0x6f, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x55, 0x0a, 0x08, 0x63, 0x6c, 0x75, 0x73, 0x74, + 0x65, 0x72, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x08, 0x63, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x32, 0x0a, 0x13, 0x61, 0x63, + 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5b, 0x0a, 0x04, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, + 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x2e, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x29, 0x0a, 0x0e, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f, 0x74, + 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x65, 0x63, 0x75, 0x72, + 0x69, 0x74, 0x79, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x32, 0x0a, 0x13, 0x69, + 0x73, 0x5f, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x69, 0x73, 0x47, 0x6c, 0x6f, 0x62, 0x61, + 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5e, 0x0a, + 0x16, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x5f, 0x61, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, + 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x76, 0x31, 0x2e, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x14, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, 0x14, 0x68, 0x69, 0x73, 0x74, 0x6f, + 0x72, 0x79, 0x5f, 0x61, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x5f, 0x75, 0x72, 0x69, 0x18, 0x0b, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x41, 0x72, 0x63, 0x68, + 0x69, 0x76, 0x61, 0x6c, 0x55, 0x72, 0x69, 0x42, 0x02, 0x68, 0x00, 0x12, 0x64, 0x0a, 0x19, 0x76, 0x69, + 0x73, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x61, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, + 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x76, 0x31, 0x2e, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x17, 0x76, 0x69, 0x73, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x41, 0x72, 0x63, 0x68, 0x69, + 0x76, 0x61, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x17, 0x76, + 0x69, 0x73, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x61, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, + 0x6c, 0x5f, 0x75, 0x72, 0x69, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x15, 0x76, 0x69, 0x73, 0x69, + 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x55, 0x72, 0x69, + 0x42, 0x02, 0x68, 0x00, 0x1a, 0x3f, 0x0a, 0x09, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x1b, 0x0a, 0x19, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xbf, 0x01, 0x0a, + 0x15, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x09, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, + 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, + 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, + 0x0a, 0x10, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, + 0x52, 0x0f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, + 0x42, 0x02, 0x68, 0x00, 0x22, 0xa4, 0x01, 0x0a, 0x16, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5e, 0x0a, + 0x0a, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x3a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0a, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, + 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x50, 0x0a, 0x18, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x12, 0x0a, 0x02, + 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, + 0xe3, 0x03, 0x0a, 0x19, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x53, 0x0a, 0x0e, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0d, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x46, 0x0a, 0x06, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x6a, 0x0a, 0x12, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x52, 0x11, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x66, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, + 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, + 0x66, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x32, 0x0a, 0x13, 0x69, 0x73, 0x5f, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x69, + 0x73, 0x47, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x5a, 0x0a, 0x10, 0x66, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x5f, + 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x72, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, + 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x0f, 0x66, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, + 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0xcf, 0x03, 0x0a, 0x16, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x53, 0x0a, 0x0b, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x6e, + 0x66, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x46, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x42, 0x02, 0x68, 0x00, 0x12, 0x6a, 0x0a, 0x12, 0x72, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x11, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, + 0x0a, 0x0e, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x54, 0x6f, + 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2e, 0x0a, 0x11, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x5f, 0x62, 0x61, 0x64, 0x5f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x42, 0x61, 0x64, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x2f, 0x0a, 0x11, 0x70, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, + 0x70, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x85, 0x03, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x53, 0x0a, + 0x0e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0d, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x46, 0x0a, + 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x6a, 0x0a, 0x12, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x72, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x52, 0x11, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x66, 0x61, 0x69, 0x6c, 0x6f, + 0x76, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0f, 0x66, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x32, 0x0a, 0x13, 0x69, 0x73, 0x5f, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x11, 0x69, 0x73, 0x47, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x68, 0x0a, 0x19, 0x44, 0x65, 0x70, 0x72, 0x65, 0x63, 0x61, 0x74, + 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x29, 0x0a, 0x0e, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x6f, 0x6b, + 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, + 0x79, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x1c, 0x0a, 0x1a, 0x44, 0x65, 0x70, + 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xb8, 0x0d, 0x0a, 0x1d, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, + 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, + 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, + 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x5b, 0x0a, 0x1a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x18, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, 0x14, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, + 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x51, 0x0a, 0x15, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, + 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x69, 0x0a, 0x18, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, + 0x64, 0x5f, 0x72, 0x65, 0x75, 0x73, 0x65, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x0b, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x52, 0x65, 0x75, 0x73, 0x65, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, + 0x15, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x52, 0x65, 0x75, 0x73, 0x65, 0x50, + 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x72, 0x0a, 0x1b, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x5f, + 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x16, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2f, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x43, 0x6f, 0x6e, 0x66, + 0x6c, 0x69, 0x63, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x18, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x50, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x0c, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x70, + 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0b, 0x72, + 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, 0x0a, + 0x0d, 0x63, 0x72, 0x6f, 0x6e, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x18, 0x0d, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x72, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x18, 0x0e, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x6d, 0x6f, 0x52, 0x04, + 0x6d, 0x65, 0x6d, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, 0x11, 0x73, 0x65, 0x61, 0x72, 0x63, + 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x0f, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x10, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, + 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x17, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x5f, 0x65, 0x61, 0x67, 0x65, 0x72, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x45, 0x61, + 0x67, 0x65, 0x72, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x51, 0x0a, 0x11, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x64, 0x5f, 0x66, 0x61, 0x69, 0x6c, + 0x75, 0x72, 0x65, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x10, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, + 0x75, 0x65, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5a, 0x0a, + 0x16, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x14, 0x6c, 0x61, 0x73, + 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, 0x14, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x72, 0x74, 0x44, 0x65, 0x6c, 0x61, 0x79, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x57, 0x0a, 0x14, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, + 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x73, 0x18, 0x15, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x52, 0x13, 0x63, + 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x0d, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x6d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x2e, + 0x55, 0x73, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x0c, 0x75, 0x73, 0x65, + 0x72, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, 0x36, 0x0a, 0x05, + 0x6c, 0x69, 0x6e, 0x6b, 0x73, 0x18, 0x18, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x6e, 0x6b, 0x52, 0x05, 0x6c, 0x69, 0x6e, 0x6b, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x22, 0xcd, 0x01, 0x0a, 0x1e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x19, 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x72, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x72, 0x0a, 0x13, 0x65, 0x61, 0x67, 0x65, + 0x72, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6f, 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x52, 0x11, 0x65, 0x61, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, + 0x73, 0x6b, 0x42, 0x02, 0x68, 0x00, 0x22, 0xb0, 0x03, 0x0a, 0x22, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x69, 0x73, + 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x09, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x2e, 0x0a, 0x11, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x5f, 0x70, + 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x6d, + 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x50, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, + 0x6b, 0x65, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, + 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x28, 0x0a, 0x0e, 0x77, 0x61, + 0x69, 0x74, 0x5f, 0x6e, 0x65, 0x77, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x0c, 0x77, 0x61, 0x69, 0x74, 0x4e, 0x65, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x6c, 0x0a, 0x19, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x5f, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x69, 0x73, 0x74, 0x6f, + 0x72, 0x79, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x79, 0x70, 0x65, + 0x52, 0x16, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x46, 0x69, 0x6c, + 0x74, 0x65, 0x72, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, 0x0a, 0x0d, 0x73, 0x6b, + 0x69, 0x70, 0x5f, 0x61, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0c, 0x73, 0x6b, 0x69, 0x70, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x61, 0x6c, 0x42, 0x02, 0x68, + 0x00, 0x22, 0xf8, 0x01, 0x0a, 0x23, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3e, 0x0a, 0x07, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, + 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x07, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x45, 0x0a, 0x0b, 0x72, 0x61, 0x77, 0x5f, 0x68, 0x69, 0x73, 0x74, 0x6f, + 0x72, 0x79, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x44, 0x61, 0x74, 0x61, 0x42, 0x6c, 0x6f, 0x62, 0x52, 0x0a, 0x72, 0x61, 0x77, 0x48, 0x69, 0x73, 0x74, + 0x6f, 0x72, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, + 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, + 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x1e, 0x0a, 0x08, 0x61, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x08, 0x61, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xf6, + 0x01, 0x0a, 0x29, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x76, 0x65, + 0x72, 0x73, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x09, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x2e, 0x0a, 0x11, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x5f, 0x70, 0x61, + 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x6d, 0x61, + 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x50, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, + 0x65, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, + 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x98, 0x01, 0x0a, 0x2a, 0x47, 0x65, + 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3e, 0x0a, 0x07, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x48, + 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x07, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, + 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, + 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xcd, 0x02, 0x0a, + 0x1c, 0x50, 0x6f, 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, + 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0a, 0x74, + 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, + 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, + 0x75, 0x65, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2b, + 0x0a, 0x0f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x43, 0x68, 0x65, + 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x42, 0x02, 0x68, 0x00, 0x12, 0x75, 0x0a, 0x1b, 0x77, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x61, 0x70, 0x61, 0x62, + 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x43, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x52, 0x19, 0x77, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x70, 0x61, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0xe7, 0x08, 0x0a, 0x1d, 0x50, 0x6f, + 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, + 0x75, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, + 0x6b, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x74, 0x61, + 0x73, 0x6b, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, + 0x0a, 0x19, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, + 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x16, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x30, 0x0a, 0x12, 0x62, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x5f, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x5f, 0x68, 0x69, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x62, 0x61, + 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x48, 0x69, 0x6e, 0x74, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, + 0x79, 0x52, 0x07, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, + 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, + 0x09, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, + 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x6b, 0x0a, 0x1d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, + 0x75, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x1a, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, + 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x45, 0x0a, 0x0e, 0x73, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0d, 0x73, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, + 0x0c, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0d, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0b, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x69, 0x0a, 0x07, + 0x71, 0x75, 0x65, 0x72, 0x69, 0x65, 0x73, 0x18, 0x0e, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4b, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6f, + 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, + 0x75, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x69, 0x65, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x71, 0x75, 0x65, 0x72, 0x69, 0x65, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x41, 0x0a, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x18, 0x0f, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x52, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x1a, 0x68, 0x0a, 0x0c, 0x51, 0x75, 0x65, 0x72, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xb6, 0x08, 0x0a, 0x23, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x21, 0x0a, 0x0a, 0x74, + 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, + 0x74, 0x61, 0x73, 0x6b, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x40, 0x0a, 0x08, + 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x08, 0x63, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x65, 0x0a, 0x11, 0x73, 0x74, 0x69, 0x63, + 0x6b, 0x79, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x69, + 0x63, 0x6b, 0x79, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x10, 0x73, 0x74, 0x69, 0x63, 0x6b, 0x79, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3b, 0x0a, 0x18, 0x72, 0x65, + 0x74, 0x75, 0x72, 0x6e, 0x5f, 0x6e, 0x65, 0x77, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x72, 0x65, 0x74, 0x75, + 0x72, 0x6e, 0x4e, 0x65, 0x77, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x46, 0x0a, 0x1e, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x5f, 0x63, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x5f, 0x6e, 0x65, 0x77, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x74, 0x61, 0x73, 0x6b, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x1a, 0x66, 0x6f, 0x72, 0x63, 0x65, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4e, 0x65, 0x77, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x54, 0x61, 0x73, 0x6b, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2b, 0x0a, 0x0f, 0x62, 0x69, 0x6e, 0x61, 0x72, + 0x79, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0e, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x7f, 0x0a, 0x0d, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x56, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, + 0x65, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x60, 0x0a, 0x14, + 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x6d, 0x70, 0x52, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x53, 0x74, 0x61, 0x6d, 0x70, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x08, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, + 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x08, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, 0x0c, 0x73, 0x64, 0x6b, 0x5f, + 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, + 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, + 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x0b, + 0x73, 0x64, 0x6b, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, + 0x0a, 0x11, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x4d, 0x65, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, + 0x10, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x42, 0x02, 0x68, 0x00, 0x1a, 0x73, 0x0a, 0x11, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x44, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0xb3, 0x02, 0x0a, 0x24, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x67, 0x0a, 0x0d, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x50, 0x6f, 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x51, + 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x42, 0x02, 0x68, 0x00, 0x12, 0x69, 0x0a, 0x0e, + 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6f, 0x6c, 0x6c, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, + 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x52, 0x0d, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x37, 0x0a, 0x16, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x68, 0x69, 0x73, 0x74, 0x6f, + 0x72, 0x79, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x13, 0x72, 0x65, 0x73, 0x65, 0x74, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd8, 0x03, 0x0a, 0x20, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, + 0x69, 0x6c, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, + 0x73, 0x6b, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x74, + 0x61, 0x73, 0x6b, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x48, 0x0a, 0x05, 0x63, + 0x61, 0x75, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, + 0x65, 0x64, 0x43, 0x61, 0x75, 0x73, 0x65, 0x52, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, + 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2b, 0x0a, 0x0f, + 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x43, 0x68, 0x65, 0x63, 0x6b, + 0x73, 0x75, 0x6d, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x76, + 0x31, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x55, 0x0a, 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x23, 0x0a, 0x21, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, + 0x69, 0x6c, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x82, 0x03, 0x0a, 0x1c, + 0x50, 0x6f, 0x6c, 0x6c, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x51, + 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0a, 0x74, 0x61, + 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, + 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, + 0x65, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x60, + 0x0a, 0x13, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x6d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x11, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x4d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, 0x75, 0x0a, 0x1b, 0x77, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x61, 0x70, 0x61, + 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x43, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x52, 0x19, 0x77, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x70, 0x61, 0x62, + 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0xb4, 0x09, 0x0a, 0x1d, 0x50, + 0x6f, 0x6c, 0x6c, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x51, + 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0a, 0x74, + 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, + 0x74, 0x61, 0x73, 0x6b, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x31, 0x0a, 0x12, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, + 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x4d, 0x0a, 0x0d, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x79, 0x70, 0x65, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x61, 0x63, 0x74, 0x69, 0x76, + 0x69, 0x74, 0x79, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x61, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, + 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, + 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x08, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x73, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x51, 0x0a, 0x11, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x5f, 0x64, 0x65, 0x74, 0x61, + 0x69, 0x6c, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x10, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, + 0x65, 0x61, 0x74, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x45, 0x0a, + 0x0e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0a, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0d, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x63, 0x0a, 0x1e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x6d, + 0x70, 0x74, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x52, 0x1b, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x53, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x41, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, + 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x1c, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x05, 0x52, + 0x07, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x58, 0x0a, 0x19, 0x73, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x16, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, + 0x6f, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x52, 0x0a, 0x16, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6c, 0x6f, 0x73, + 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, + 0x6f, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x4a, 0x0a, 0x11, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x6f, 0x75, 0x74, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x10, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x54, 0x69, 0x6d, + 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x0c, 0x72, 0x65, 0x74, 0x72, 0x79, + 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, + 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, + 0xc9, 0x01, 0x0a, 0x22, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, + 0x79, 0x54, 0x61, 0x73, 0x6b, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x6f, 0x6b, 0x65, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x54, 0x6f, 0x6b, 0x65, + 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x22, 0x54, 0x0a, 0x23, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, + 0x79, 0x54, 0x61, 0x73, 0x6b, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2d, 0x0a, 0x10, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x5f, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x63, + 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x8f, 0x02, 0x0a, 0x26, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, + 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, + 0x42, 0x79, 0x49, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x19, 0x0a, + 0x06, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x72, 0x75, + 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, + 0x74, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x61, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x79, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x64, 0x65, 0x74, + 0x61, 0x69, 0x6c, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, + 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, + 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, + 0x42, 0x02, 0x68, 0x00, 0x22, 0x58, 0x0a, 0x27, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x41, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, + 0x61, 0x74, 0x42, 0x79, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2d, 0x0a, + 0x10, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x9f, 0x02, 0x0a, 0x23, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, + 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0c, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x3c, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, + 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x55, 0x0a, 0x0e, 0x77, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x65, + 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x26, 0x0a, 0x24, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, + 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x8e, 0x02, 0x0a, 0x27, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, + 0x42, 0x79, 0x49, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x19, + 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x72, + 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x61, 0x63, 0x74, 0x69, 0x76, + 0x69, 0x74, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x61, 0x63, 0x74, + 0x69, 0x76, 0x69, 0x74, 0x79, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3c, 0x0a, 0x06, 0x72, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x2a, 0x0a, 0x28, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, + 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x64, 0x42, 0x79, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xfa, 0x02, 0x0a, + 0x20, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, + 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x52, 0x07, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, + 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, + 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5a, 0x0a, 0x16, 0x6c, + 0x61, 0x73, 0x74, 0x5f, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x5f, 0x64, 0x65, 0x74, + 0x61, 0x69, 0x6c, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x48, + 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x55, 0x0a, 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x74, + 0x61, 0x6d, 0x70, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x65, 0x0a, 0x21, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, + 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x40, 0x0a, 0x08, 0x66, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x08, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0xe9, 0x02, 0x0a, 0x24, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, + 0x65, 0x64, 0x42, 0x79, 0x49, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, + 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x19, 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x72, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x61, 0x63, 0x74, + 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x61, + 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, + 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, + 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x07, 0x66, 0x61, + 0x69, 0x6c, 0x75, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x74, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5a, 0x0a, 0x16, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x68, + 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x5f, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, + 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x48, 0x65, 0x61, 0x72, 0x74, + 0x62, 0x65, 0x61, 0x74, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x69, + 0x0a, 0x25, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, + 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x42, 0x79, 0x49, 0x64, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x40, 0x0a, 0x08, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x52, 0x08, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x22, 0xa0, 0x02, 0x0a, 0x22, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, + 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, + 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, + 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, + 0x69, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, + 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, + 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x55, 0x0a, 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x25, 0x0a, 0x23, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, + 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, + 0x6c, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x8f, 0x02, 0x0a, 0x26, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, + 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x42, 0x79, 0x49, 0x64, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x19, 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x72, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x23, 0x0a, 0x0b, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x73, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x29, + 0x0a, 0x27, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, + 0x54, 0x61, 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x42, 0x79, 0x49, 0x64, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xbf, 0x02, 0x0a, 0x25, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, + 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, + 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, + 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x37, 0x0a, 0x16, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x66, 0x69, 0x72, 0x73, 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1a, 0x0a, 0x06, + 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, + 0x73, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x28, 0x0a, 0x26, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xe1, + 0x03, 0x0a, 0x1e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x73, 0x69, 0x67, 0x6e, + 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x69, + 0x67, 0x6e, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x69, + 0x6e, 0x70, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x3a, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x1b, 0x73, 0x6b, 0x69, 0x70, + 0x5f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x73, 0x6b, 0x69, + 0x70, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x54, 0x61, 0x73, 0x6b, 0x42, 0x02, 0x68, 0x00, 0x22, 0x21, 0x0a, 0x1f, 0x53, 0x69, 0x67, 0x6e, 0x61, + 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x95, 0x0c, 0x0a, 0x27, 0x53, 0x69, 0x67, + 0x6e, 0x61, 0x6c, 0x57, 0x69, 0x74, 0x68, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, + 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x47, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, + 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x73, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5b, 0x0a, + 0x1a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x18, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, 0x14, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x51, 0x0a, 0x15, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, + 0x73, 0x6b, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x09, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x0a, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x69, 0x0a, 0x18, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x5f, 0x72, + 0x65, 0x75, 0x73, 0x65, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, + 0x64, 0x52, 0x65, 0x75, 0x73, 0x65, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x15, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x52, 0x65, 0x75, 0x73, 0x65, 0x50, 0x6f, 0x6c, 0x69, 0x63, + 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x72, 0x0a, 0x1b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x69, 0x64, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x5f, 0x70, 0x6f, 0x6c, 0x69, + 0x63, 0x79, 0x18, 0x16, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, + 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x18, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x49, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x4e, 0x61, + 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0c, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x5f, + 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x0b, 0x73, 0x69, 0x67, 0x6e, + 0x61, 0x6c, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, 0x0a, 0x07, 0x63, 0x6f, + 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, + 0x72, 0x6f, 0x6c, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x0c, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, + 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0b, + 0x72, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, + 0x0a, 0x0d, 0x63, 0x72, 0x6f, 0x6e, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x18, + 0x10, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x72, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, + 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x18, 0x11, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x6d, 0x6f, 0x52, + 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, 0x11, 0x73, 0x65, 0x61, 0x72, + 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x12, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x10, 0x73, 0x65, 0x61, 0x72, 0x63, + 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, + 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, 0x14, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x18, 0x14, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x72, 0x74, 0x44, 0x65, 0x6c, 0x61, 0x79, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x1b, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x67, 0x65, 0x6e, 0x65, + 0x72, 0x61, 0x74, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, + 0x6b, 0x18, 0x15, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x73, 0x6b, 0x69, 0x70, 0x47, 0x65, 0x6e, 0x65, + 0x72, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x4a, 0x0a, 0x0d, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x73, + 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x0c, 0x75, 0x73, 0x65, 0x72, 0x4d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x22, 0x63, 0x0a, 0x28, 0x53, 0x69, + 0x67, 0x6e, 0x61, 0x6c, 0x57, 0x69, 0x74, 0x68, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x19, 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x72, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1c, + 0x0a, 0x07, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xf2, 0x03, 0x0a, 0x1d, 0x52, + 0x65, 0x73, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x1a, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x44, 0x0a, + 0x1d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x66, + 0x69, 0x6e, 0x69, 0x73, 0x68, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x19, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, + 0x46, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x59, 0x0a, 0x12, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x61, 0x70, 0x70, 0x6c, + 0x79, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x54, 0x79, + 0x70, 0x65, 0x52, 0x10, 0x72, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x54, + 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x71, 0x0a, 0x1b, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, + 0x72, 0x65, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x5f, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, + 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x45, 0x78, 0x63, + 0x6c, 0x75, 0x64, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x18, 0x72, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, + 0x61, 0x70, 0x70, 0x6c, 0x79, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, 0x79, 0x70, 0x65, 0x73, + 0x42, 0x02, 0x68, 0x00, 0x22, 0x3b, 0x0a, 0x1e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x19, 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x72, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xd8, 0x02, + 0x0a, 0x21, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1a, 0x0a, 0x06, 0x72, 0x65, + 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, + 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x37, 0x0a, 0x16, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x66, + 0x69, 0x72, 0x73, 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x75, 0x6e, 0x49, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x24, 0x0a, 0x22, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, + 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa0, 0x01, 0x0a, 0x1e, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x21, 0x0a, 0x1f, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xba, 0x03, 0x0a, 0x21, 0x4c, 0x69, 0x73, 0x74, 0x4f, 0x70, 0x65, 0x6e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2e, 0x0a, 0x11, 0x6d, 0x61, 0x78, 0x69, + 0x6d, 0x75, 0x6d, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x0f, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x50, 0x61, 0x67, 0x65, 0x53, 0x69, + 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, + 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, + 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x57, 0x0a, 0x11, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x66, 0x69, 0x6c, + 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x76, 0x31, + 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, + 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x60, 0x0a, 0x10, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, + 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x48, 0x00, 0x52, 0x0f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x51, 0x0a, 0x0b, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x48, 0x00, 0x52, + 0x0a, 0x74, 0x79, 0x70, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x42, 0x09, + 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x22, 0xa5, 0x01, 0x0a, 0x22, 0x4c, 0x69, 0x73, + 0x74, 0x4f, 0x70, 0x65, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x53, 0x0a, + 0x0a, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, + 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, + 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8d, 0x04, 0x0a, 0x23, 0x4c, 0x69, 0x73, 0x74, + 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2e, 0x0a, + 0x11, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x50, + 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, + 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x57, 0x0a, 0x11, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x69, 0x6c, 0x74, + 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x46, + 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x46, + 0x69, 0x6c, 0x74, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x60, 0x0a, 0x10, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x48, + 0x00, 0x52, 0x0f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, 0x51, 0x0a, 0x0b, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x66, 0x69, 0x6c, + 0x74, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x76, 0x31, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x46, 0x69, 0x6c, 0x74, + 0x65, 0x72, 0x48, 0x00, 0x52, 0x0a, 0x74, 0x79, 0x70, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, 0x0d, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x5f, 0x66, 0x69, 0x6c, + 0x74, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x76, + 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x48, 0x00, 0x52, + 0x0c, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, + 0x42, 0x09, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x22, 0xa7, 0x01, 0x0a, 0x24, 0x4c, + 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x53, 0x0a, 0x0a, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x0a, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, + 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, + 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xa8, 0x01, 0x0a, 0x1d, 0x4c, + 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1f, 0x0a, 0x09, 0x70, + 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, + 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, + 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0xa1, 0x01, 0x0a, + 0x1e, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x53, 0x0a, + 0x0a, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, + 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, + 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xb0, 0x01, 0x0a, 0x25, 0x4c, 0x69, 0x73, 0x74, + 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x1f, 0x0a, 0x09, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x08, 0x70, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, + 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, + 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x42, 0x02, 0x68, 0x00, + 0x22, 0xa9, 0x01, 0x0a, 0x26, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, 0x64, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x53, 0x0a, 0x0a, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, + 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x22, 0xa8, 0x01, 0x0a, 0x1d, 0x53, 0x63, 0x61, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x1f, 0x0a, 0x09, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, + 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, + 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0xa1, 0x01, 0x0a, 0x1e, 0x53, 0x63, 0x61, 0x6e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x53, 0x0a, 0x0a, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, + 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, + 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, + 0x5c, 0x0a, 0x1e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x18, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0xa0, 0x02, 0x0a, 0x1f, 0x43, 0x6f, 0x75, + 0x6e, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x05, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x6d, 0x0a, 0x06, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x51, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x47, 0x72, 0x6f, 0x75, 0x70, + 0x52, 0x06, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x74, 0x0a, 0x10, 0x41, + 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, + 0x46, 0x0a, 0x0c, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x52, 0x0b, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x22, 0x1c, 0x0a, 0x1a, 0x47, + 0x65, 0x74, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xe7, 0x01, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x53, + 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5e, 0x0a, 0x04, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4b, 0x65, + 0x79, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x6b, 0x65, 0x79, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x1a, 0x68, 0x0a, 0x09, 0x4b, 0x65, 0x79, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x41, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x27, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, + 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x64, 0x56, 0x61, + 0x6c, 0x75, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xb2, 0x02, 0x0a, 0x20, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, + 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, + 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x51, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x26, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, + 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, + 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0c, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x0b, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x27, 0x0a, 0x0d, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x4a, 0x04, 0x08, 0x05, 0x10, 0x06, 0x22, 0x23, 0x0a, 0x21, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x64, 0x51, 0x75, 0x65, 0x72, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, + 0x65, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x8c, 0x01, 0x0a, 0x1b, + 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x74, 0x69, 0x63, 0x6b, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, + 0x65, 0x75, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x09, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x1e, 0x0a, 0x1c, 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x74, 0x69, 0x63, 0x6b, + 0x79, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0xac, 0x02, 0x0a, 0x14, 0x51, 0x75, 0x65, 0x72, 0x79, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x09, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x51, + 0x75, 0x65, 0x72, 0x79, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x65, + 0x0a, 0x16, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x72, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x63, 0x6f, + 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2b, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, + 0x2e, 0x76, 0x31, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x43, 0x6f, + 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x14, 0x71, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x6a, + 0x65, 0x63, 0x74, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, + 0xb1, 0x01, 0x0a, 0x15, 0x51, 0x75, 0x65, 0x72, 0x79, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x47, 0x0a, 0x0c, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x52, 0x0b, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, + 0x0e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x72, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x76, 0x31, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x52, 0x65, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x0d, 0x71, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, + 0x6a, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0x91, 0x01, 0x0a, 0x20, 0x44, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x09, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xe0, 0x05, 0x0a, 0x21, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x60, 0x0a, 0x10, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x52, 0x0f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x42, 0x02, 0x68, 0x00, 0x12, 0x6b, 0x0a, 0x17, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x15, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x60, 0x0a, 0x12, + 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, + 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x11, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x41, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x69, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x62, 0x0a, 0x10, 0x70, 0x65, 0x6e, + 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x18, 0x04, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x65, 0x6e, 0x64, + 0x69, 0x6e, 0x67, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0f, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x68, 0x69, 0x6c, + 0x64, 0x72, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x69, 0x0a, 0x15, 0x70, 0x65, 0x6e, 0x64, 0x69, + 0x6e, 0x67, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x50, + 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, + 0x73, 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x13, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x42, 0x02, 0x68, 0x00, 0x12, 0x48, 0x0a, + 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x26, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, + 0x6b, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x71, 0x0a, 0x18, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x6e, 0x65, + 0x78, 0x75, 0x73, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x07, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x65, 0x6e, + 0x64, 0x69, 0x6e, 0x67, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x16, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x4e, 0x65, 0x78, + 0x75, 0x73, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, + 0x9d, 0x05, 0x0a, 0x18, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x51, + 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0a, 0x74, + 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, + 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, + 0x75, 0x65, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x50, 0x0a, 0x0f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x54, + 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0d, 0x74, 0x61, 0x73, + 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, + 0x19, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, + 0x75, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x16, + 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x08, 0x61, 0x70, 0x69, + 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, + 0x31, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, + 0x75, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x07, 0x61, 0x70, 0x69, 0x4d, 0x6f, 0x64, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x54, 0x0a, 0x08, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, + 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x52, 0x0a, 0x10, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, + 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, + 0x0e, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x54, 0x79, 0x70, 0x65, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x73, 0x74, 0x61, + 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6f, 0x72, + 0x74, 0x5f, 0x70, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, + 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x3c, 0x0a, 0x18, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x72, + 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x16, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x61, 0x63, 0x68, + 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0xad, 0x03, 0x0a, 0x19, 0x44, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x43, 0x0a, 0x07, 0x70, 0x6f, 0x6c, 0x6c, 0x65, 0x72, + 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x70, 0x6f, 0x6c, + 0x6c, 0x65, 0x72, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x5a, 0x0a, 0x11, 0x74, 0x61, 0x73, 0x6b, 0x5f, + 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, + 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x0f, 0x74, 0x61, 0x73, + 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x75, 0x0a, 0x0d, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x61, 0x73, + 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x1a, + 0x78, 0x0a, 0x11, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x49, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x17, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xbb, 0x04, 0x0a, 0x16, 0x47, + 0x65, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7e, 0x0a, 0x11, 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, + 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4d, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, + 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x43, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, + 0x65, 0x64, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, 0x0a, + 0x0e, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, + 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4b, 0x0a, 0x0c, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x0b, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x25, 0x0a, 0x0c, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x32, 0x0a, 0x13, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x5f, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x52, + 0x11, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x43, 0x6f, 0x75, 0x6e, + 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2f, 0x0a, 0x11, 0x70, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, + 0x65, 0x6e, 0x63, 0x65, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x10, 0x70, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x6f, 0x72, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x76, 0x69, 0x73, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, + 0x79, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x76, 0x69, + 0x73, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x1a, 0x4b, 0x0a, 0x15, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x43, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x16, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, + 0x79, 0x73, 0x74, 0x65, 0x6d, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, + 0xb0, 0x06, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x0a, 0x0e, 0x73, 0x65, 0x72, 0x76, 0x65, + 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, + 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x6b, 0x0a, 0x0c, 0x63, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x43, 0x61, 0x70, 0x61, 0x62, + 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x52, 0x0c, 0x63, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, + 0x74, 0x69, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x1a, 0xfe, 0x04, 0x0a, 0x0c, 0x43, 0x61, 0x70, 0x61, + 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x12, 0x39, 0x0a, 0x17, 0x73, 0x69, 0x67, 0x6e, 0x61, + 0x6c, 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x14, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x41, 0x6e, + 0x64, 0x51, 0x75, 0x65, 0x72, 0x79, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x48, 0x0a, 0x1e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, + 0x5f, 0x64, 0x69, 0x66, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x1c, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x45, 0x72, + 0x72, 0x6f, 0x72, 0x44, 0x69, 0x66, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, 0x22, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, + 0x79, 0x5f, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, + 0x5f, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x1f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x49, + 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x31, 0x0a, 0x12, 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x5f, 0x73, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x73, 0x75, + 0x70, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x40, 0x0a, 0x1a, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x65, 0x64, 0x5f, 0x66, 0x61, 0x69, + 0x6c, 0x75, 0x72, 0x65, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x65, 0x64, 0x46, 0x61, 0x69, 0x6c, + 0x75, 0x72, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x3d, 0x0a, 0x19, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x5f, 0x62, 0x61, 0x73, 0x65, + 0x64, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x16, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x61, 0x73, 0x65, 0x64, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, + 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0a, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x4d, 0x65, 0x6d, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x34, 0x0a, 0x14, 0x65, 0x61, 0x67, 0x65, 0x72, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x65, 0x61, 0x67, + 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x72, 0x74, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x25, 0x0a, 0x0c, 0x73, 0x64, 0x6b, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x64, 0x6b, 0x4d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x42, 0x02, 0x68, 0x00, 0x12, 0x48, 0x0a, 0x1f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, + 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x1b, 0x63, + 0x6f, 0x75, 0x6e, 0x74, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, + 0x6e, 0x65, 0x78, 0x75, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x6e, 0x65, 0x78, 0x75, + 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8b, 0x01, 0x0a, 0x1e, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x61, + 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, + 0x75, 0x65, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x09, 0x74, + 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xa1, 0x02, 0x0a, 0x1f, + 0x4c, 0x69, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7e, 0x0a, + 0x1e, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, + 0x65, 0x75, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, + 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x1b, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, + 0x79, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x7e, 0x0a, 0x1e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, + 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x52, 0x1b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, + 0x75, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, + 0xc8, 0x03, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x73, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x42, 0x0a, 0x08, 0x73, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x08, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x50, 0x0a, 0x0d, 0x69, + 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x70, 0x61, 0x74, 0x63, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x27, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x63, 0x68, 0x52, 0x0c, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, + 0x50, 0x61, 0x74, 0x63, 0x68, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x74, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x18, + 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, + 0x6d, 0x6f, 0x52, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, 0x11, 0x73, + 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x61, + 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x10, 0x73, 0x65, + 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x43, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, + 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, + 0x66, 0x6c, 0x69, 0x63, 0x74, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x60, 0x0a, 0x17, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, + 0xda, 0x02, 0x0a, 0x18, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x53, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x08, 0x73, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, + 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x08, 0x73, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3e, 0x0a, 0x04, 0x69, 0x6e, + 0x66, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x04, 0x69, + 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x34, 0x0a, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x6d, 0x6f, + 0x52, 0x04, 0x6d, 0x65, 0x6d, 0x6f, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, 0x11, 0x73, 0x65, + 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x61, 0x72, + 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x10, 0x73, 0x65, 0x61, + 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x29, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x5f, 0x74, 0x6f, 0x6b, 0x65, + 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, + 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xeb, 0x02, 0x0a, 0x15, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x42, 0x0a, 0x08, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x08, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, + 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x63, 0x6f, 0x6e, + 0x66, 0x6c, 0x69, 0x63, 0x74, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, + 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, + 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x59, 0x0a, 0x11, + 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, + 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x10, 0x73, + 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x22, 0x18, 0x0a, 0x16, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xe3, 0x01, 0x0a, 0x14, + 0x50, 0x61, 0x74, 0x63, 0x68, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x41, 0x0a, 0x05, 0x70, 0x61, 0x74, 0x63, 0x68, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x63, 0x68, 0x52, 0x05, 0x70, 0x61, 0x74, 0x63, + 0x68, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, + 0x42, 0x02, 0x68, 0x00, 0x22, 0x17, 0x0a, 0x15, 0x50, 0x61, 0x74, 0x63, 0x68, 0x53, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xe3, 0x01, 0x0a, + 0x20, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4d, 0x61, 0x74, 0x63, + 0x68, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x23, 0x0a, 0x0b, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, + 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x39, 0x0a, + 0x08, 0x65, 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x07, 0x65, 0x6e, 0x64, 0x54, 0x69, 0x6d, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0x62, 0x0a, 0x21, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x22, 0x7e, 0x0a, 0x15, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, 0x0b, 0x73, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0a, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, + 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x18, 0x0a, 0x16, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xae, 0x01, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2e, 0x0a, + 0x11, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x50, + 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, + 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x92, 0x01, 0x0a, + 0x15, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4d, 0x0a, 0x09, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x09, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, + 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, + 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x9c, 0x07, 0x0a, 0x27, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x4f, 0x0a, 0x23, 0x61, 0x64, 0x64, 0x5f, 0x6e, 0x65, 0x77, 0x5f, 0x62, + 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x5f, 0x69, 0x6e, 0x5f, 0x6e, 0x65, 0x77, 0x5f, 0x64, 0x65, + 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x73, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, + 0x52, 0x1c, 0x61, 0x64, 0x64, 0x4e, 0x65, 0x77, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x49, 0x6e, + 0x4e, 0x65, 0x77, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x53, 0x65, 0x74, 0x42, 0x02, 0x68, + 0x00, 0x12, 0xa4, 0x01, 0x0a, 0x1b, 0x61, 0x64, 0x64, 0x5f, 0x6e, 0x65, 0x77, 0x5f, 0x63, 0x6f, 0x6d, + 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x60, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, + 0x72, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x64, 0x64, 0x4e, 0x65, + 0x77, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x48, 0x00, 0x52, 0x17, 0x61, 0x64, 0x64, 0x4e, 0x65, 0x77, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, + 0x69, 0x62, 0x6c, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, + 0x0a, 0x17, 0x70, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x5f, 0x73, 0x65, 0x74, 0x5f, 0x62, 0x79, 0x5f, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, + 0x13, 0x70, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x65, 0x74, 0x42, 0x79, 0x42, 0x75, 0x69, 0x6c, + 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x42, 0x0a, 0x1b, 0x70, 0x72, 0x6f, 0x6d, 0x6f, 0x74, + 0x65, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x69, + 0x6e, 0x5f, 0x73, 0x65, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x17, 0x70, 0x72, + 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x57, 0x69, 0x74, 0x68, 0x69, + 0x6e, 0x53, 0x65, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x77, 0x0a, 0x0a, 0x6d, 0x65, 0x72, 0x67, 0x65, + 0x5f, 0x73, 0x65, 0x74, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x52, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x43, 0x6f, 0x6d, + 0x70, 0x61, 0x74, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x2e, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x53, 0x65, 0x74, 0x73, 0x48, 0x00, 0x52, 0x09, 0x6d, 0x65, 0x72, + 0x67, 0x65, 0x53, 0x65, 0x74, 0x73, 0x42, 0x02, 0x68, 0x00, 0x1a, 0xb2, 0x01, 0x0a, 0x17, 0x41, 0x64, + 0x64, 0x4e, 0x65, 0x77, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x24, 0x0a, 0x0c, 0x6e, 0x65, 0x77, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6e, 0x65, 0x77, 0x42, 0x75, 0x69, + 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x43, 0x0a, 0x1c, 0x65, 0x78, 0x69, 0x73, 0x74, + 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x5f, 0x62, + 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x65, 0x78, + 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x42, + 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2c, 0x0a, 0x10, 0x6d, 0x61, 0x6b, + 0x65, 0x5f, 0x73, 0x65, 0x74, 0x5f, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0e, 0x6d, 0x61, 0x6b, 0x65, 0x53, 0x65, 0x74, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, + 0x74, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x79, 0x0a, 0x09, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x53, 0x65, 0x74, + 0x73, 0x12, 0x33, 0x0a, 0x14, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x65, 0x74, 0x5f, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x70, + 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x53, 0x65, 0x74, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x37, 0x0a, 0x16, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x61, 0x72, 0x79, 0x5f, + 0x73, 0x65, 0x74, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x13, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x61, 0x72, 0x79, 0x53, 0x65, 0x74, 0x42, 0x75, + 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x42, 0x0b, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x40, 0x0a, 0x28, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x43, 0x6f, 0x6d, 0x70, 0x61, + 0x74, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x4a, + 0x04, 0x08, 0x01, 0x10, 0x02, 0x52, 0x0e, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x65, + 0x74, 0x5f, 0x69, 0x64, 0x22, 0x8a, 0x01, 0x0a, 0x24, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, + 0x72, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, + 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, + 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1d, 0x0a, 0x08, 0x6d, + 0x61, 0x78, 0x5f, 0x73, 0x65, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x6d, 0x61, + 0x78, 0x53, 0x65, 0x74, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8a, 0x01, 0x0a, 0x25, 0x47, 0x65, 0x74, + 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x43, 0x6f, 0x6d, 0x70, + 0x61, 0x74, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x61, 0x0a, 0x12, 0x6d, 0x61, 0x6a, 0x6f, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x5f, 0x73, 0x65, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, + 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, + 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x74, 0x52, 0x10, 0x6d, 0x61, 0x6a, 0x6f, 0x72, + 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x74, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0xc9, + 0x10, 0x0a, 0x22, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x29, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x5f, + 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x66, + 0x6c, 0x69, 0x63, 0x74, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x9b, 0x01, 0x0a, + 0x16, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, + 0x74, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x5f, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, + 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, + 0x73, 0x65, 0x72, 0x74, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, + 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x48, 0x00, 0x52, 0x14, 0x69, 0x6e, 0x73, 0x65, 0x72, + 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x9e, 0x01, 0x0a, 0x17, 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x5f, 0x61, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x60, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, + 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x48, + 0x00, 0x52, 0x15, 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x9b, 0x01, 0x0a, 0x16, 0x64, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x5f, + 0x72, 0x75, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x5f, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x52, + 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, + 0x74, 0x52, 0x75, 0x6c, 0x65, 0x48, 0x00, 0x52, 0x14, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x41, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0xab, 0x01, 0x0a, 0x1c, 0x61, 0x64, 0x64, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, + 0x65, 0x5f, 0x72, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x64, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, + 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x64, 0x64, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, + 0x62, 0x6c, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, + 0x74, 0x52, 0x75, 0x6c, 0x65, 0x48, 0x00, 0x52, 0x19, 0x61, 0x64, 0x64, 0x43, 0x6f, 0x6d, 0x70, 0x61, + 0x74, 0x69, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0xb7, 0x01, 0x0a, 0x20, 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x5f, + 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x5f, 0x72, 0x65, 0x64, 0x69, 0x72, 0x65, + 0x63, 0x74, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x68, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, + 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x52, 0x65, + 0x70, 0x6c, 0x61, 0x63, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x42, 0x75, + 0x69, 0x6c, 0x64, 0x49, 0x64, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, + 0x65, 0x48, 0x00, 0x52, 0x1d, 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, + 0x74, 0x69, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0xb4, 0x01, 0x0a, 0x1f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, + 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x5f, 0x72, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, + 0x74, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x67, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, + 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x42, 0x75, 0x69, 0x6c, + 0x64, 0x49, 0x64, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x48, 0x00, + 0x52, 0x1c, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, + 0x65, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x7f, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, + 0x69, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x51, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x52, 0x75, + 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, + 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x48, 0x00, 0x52, 0x0d, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, + 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x8a, 0x01, 0x0a, 0x1b, 0x49, + 0x6e, 0x73, 0x65, 0x72, 0x74, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, + 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x75, 0x6c, 0x65, + 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x72, 0x75, 0x6c, + 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x02, 0x68, 0x00, 0x12, 0x48, 0x0a, 0x04, 0x72, 0x75, 0x6c, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, + 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x04, 0x72, 0x75, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x1a, + 0xa5, 0x01, 0x0a, 0x1c, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, + 0x49, 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x12, + 0x21, 0x0a, 0x0a, 0x72, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x09, 0x72, 0x75, 0x6c, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x48, 0x0a, 0x04, 0x72, 0x75, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, + 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x04, 0x72, 0x75, + 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x5a, + 0x0a, 0x1b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x41, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x21, 0x0a, 0x0a, 0x72, + 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, + 0x72, 0x75, 0x6c, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x02, 0x68, 0x00, 0x12, 0x18, 0x0a, 0x05, + 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x74, 0x0a, 0x20, 0x41, 0x64, 0x64, 0x43, 0x6f, 0x6d, 0x70, + 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x52, 0x65, 0x64, 0x69, + 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x50, 0x0a, 0x04, 0x72, 0x75, 0x6c, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, + 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x04, 0x72, 0x75, 0x6c, + 0x65, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x78, 0x0a, 0x24, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x52, + 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x50, 0x0a, 0x04, 0x72, 0x75, + 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x42, 0x75, 0x69, 0x6c, + 0x64, 0x49, 0x64, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x04, + 0x72, 0x75, 0x6c, 0x65, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x51, 0x0a, 0x23, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, + 0x49, 0x64, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x2a, 0x0a, + 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42, 0x75, 0x69, 0x6c, + 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x1a, 0x55, 0x0a, 0x0d, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, + 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x12, 0x2a, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, + 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, + 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x18, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, + 0x66, 0x6f, 0x72, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x42, 0x0b, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xc2, 0x02, 0x0a, 0x23, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x52, 0x75, + 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x6a, 0x0a, 0x10, 0x61, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x65, 0x64, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, + 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x0f, 0x61, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x83, 0x01, 0x0a, 0x19, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x5f, + 0x72, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x65, 0x64, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, + 0x6c, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, + 0x52, 0x75, 0x6c, 0x65, 0x52, 0x17, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x52, + 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x29, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x54, + 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x66, 0x0a, 0x1f, 0x47, 0x65, 0x74, 0x57, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x52, 0x75, + 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, + 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, + 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xbf, 0x02, 0x0a, 0x20, 0x47, + 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, + 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x6a, 0x0a, + 0x10, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x65, 0x64, 0x42, 0x75, 0x69, 0x6c, 0x64, + 0x49, 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x52, + 0x0f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x83, 0x01, 0x0a, 0x19, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, + 0x65, 0x5f, 0x72, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x65, 0x64, 0x43, 0x6f, 0x6d, 0x70, 0x61, + 0x74, 0x69, 0x62, 0x6c, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x52, 0x65, 0x64, 0x69, 0x72, + 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x17, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, + 0x6c, 0x65, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x29, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x5f, 0x74, 0x6f, + 0x6b, 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x66, 0x6c, 0x69, + 0x63, 0x74, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xdb, 0x01, 0x0a, 0x20, 0x47, + 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x61, 0x63, 0x68, + 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1f, 0x0a, + 0x09, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x08, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x23, 0x0a, + 0x0b, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x4f, 0x0a, 0x0c, 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, + 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x0c, 0x72, 0x65, 0x61, + 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8b, 0x01, 0x0a, + 0x21, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x61, + 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x66, 0x0a, 0x15, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x5f, 0x72, 0x65, 0x61, 0x63, + 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, + 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x52, + 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x13, 0x62, 0x75, 0x69, 0x6c, + 0x64, 0x49, 0x64, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, + 0x02, 0x68, 0x00, 0x22, 0xe1, 0x02, 0x0a, 0x1e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x5c, 0x0a, 0x12, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x37, 0x0a, 0x16, + 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, + 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x66, 0x69, 0x72, 0x73, + 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0b, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, + 0x61, 0x69, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0a, 0x77, 0x61, 0x69, 0x74, 0x50, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, + 0x02, 0x68, 0x00, 0x22, 0xfe, 0x01, 0x0a, 0x1f, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x44, 0x0a, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x72, + 0x65, 0x66, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x66, 0x52, 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x52, 0x65, 0x66, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x07, 0x6f, 0x75, 0x74, 0x63, 0x6f, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x4f, 0x75, 0x74, 0x63, 0x6f, 0x6d, 0x65, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x63, 0x6f, 0x6d, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x56, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x3c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, + 0x66, 0x65, 0x63, 0x79, 0x63, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, + 0x67, 0x65, 0x42, 0x02, 0x68, 0x00, 0x22, 0xb9, 0x06, 0x0a, 0x1a, 0x53, 0x74, 0x61, 0x72, 0x74, 0x42, + 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x2d, 0x0a, 0x10, 0x76, 0x69, 0x73, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, + 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x76, 0x69, 0x73, + 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x51, 0x75, 0x65, 0x72, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x19, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1a, 0x0a, 0x06, 0x72, 0x65, 0x61, + 0x73, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, + 0x42, 0x02, 0x68, 0x00, 0x12, 0x4d, 0x0a, 0x0a, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x0a, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, + 0x0a, 0x19, 0x6d, 0x61, 0x78, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, + 0x70, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x02, 0x52, + 0x16, 0x6d, 0x61, 0x78, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x50, 0x65, 0x72, + 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x6b, 0x0a, 0x15, 0x74, 0x65, 0x72, + 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x61, + 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x65, 0x72, 0x6d, 0x69, + 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x14, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x5c, 0x0a, 0x10, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, + 0x2e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x69, + 0x67, 0x6e, 0x61, 0x6c, 0x48, 0x00, 0x52, 0x0f, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x6e, 0x0a, 0x16, 0x63, 0x61, 0x6e, + 0x63, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, + 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, + 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x15, 0x63, 0x61, 0x6e, 0x63, 0x65, + 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x62, 0x0a, 0x12, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x11, 0x64, 0x65, + 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x59, 0x0a, 0x0f, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, + 0x2e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, + 0x73, 0x65, 0x74, 0x48, 0x00, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x42, 0x0b, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x1d, 0x0a, 0x1b, 0x53, 0x74, 0x61, 0x72, 0x74, 0x42, 0x61, 0x74, 0x63, + 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x94, 0x01, 0x0a, 0x19, 0x53, 0x74, 0x6f, 0x70, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x19, + 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, + 0x6f, 0x62, 0x49, 0x64, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1a, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, + 0x22, 0x1c, 0x0a, 0x1a, 0x53, 0x74, 0x6f, 0x70, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x5c, 0x0a, 0x1d, + 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x19, 0x0a, 0x06, 0x6a, + 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, + 0x64, 0x42, 0x02, 0x68, 0x00, 0x22, 0xc3, 0x04, 0x0a, 0x1e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x0e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x29, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, + 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0d, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x54, 0x79, 0x70, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x19, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x44, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x2a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x65, 0x6e, + 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x3d, 0x0a, 0x0a, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x63, 0x6c, + 0x6f, 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x36, 0x0a, 0x15, 0x74, 0x6f, + 0x74, 0x61, 0x6c, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x13, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x3c, 0x0a, 0x18, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x16, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3a, 0x0a, 0x17, 0x66, 0x61, 0x69, + 0x6c, 0x75, 0x72, 0x65, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, + 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x42, 0x02, + 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x09, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x1a, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0x8b, 0x01, 0x0a, 0x1a, + 0x4c, 0x69, 0x73, 0x74, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1f, 0x0a, 0x09, 0x70, 0x61, 0x67, 0x65, + 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x61, 0x67, 0x65, + 0x53, 0x69, 0x7a, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, + 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x9f, 0x01, 0x0a, 0x1b, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x54, 0x0a, 0x0e, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x61, + 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x0d, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x2a, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, + 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, + 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x22, 0xf5, 0x01, 0x0a, 0x22, 0x50, + 0x6f, 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, + 0x44, 0x0a, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x65, 0x66, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x52, 0x65, 0x66, 0x52, 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x66, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, + 0x12, 0x47, 0x0a, 0x0b, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x61, 0x69, 0x74, + 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0a, 0x77, 0x61, 0x69, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, + 0x79, 0x42, 0x02, 0x68, 0x00, 0x22, 0x82, 0x02, 0x0a, 0x23, 0x50, 0x6f, 0x6c, 0x6c, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3d, 0x0a, 0x07, 0x6f, 0x75, + 0x74, 0x63, 0x6f, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x4f, 0x75, 0x74, 0x63, 0x6f, 0x6d, 0x65, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x63, 0x6f, + 0x6d, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x56, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x67, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x3c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x4c, 0x69, 0x66, 0x65, 0x63, 0x79, 0x63, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x52, + 0x05, 0x73, 0x74, 0x61, 0x67, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x44, 0x0a, 0x0a, 0x75, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x5f, 0x72, 0x65, 0x66, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x75, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x66, 0x52, 0x09, 0x75, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x66, 0x42, 0x02, 0x68, 0x00, 0x22, 0x9d, 0x02, 0x0a, 0x19, + 0x50, 0x6f, 0x6c, 0x6c, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x74, 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x47, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, + 0x65, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x74, 0x61, 0x73, 0x6b, 0x71, 0x75, 0x65, 0x75, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x09, 0x74, + 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x75, 0x0a, 0x1b, 0x77, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x61, 0x70, + 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x52, 0x19, + 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x70, 0x61, + 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x22, 0x7d, 0x0a, 0x1a, 0x50, + 0x6f, 0x6c, 0x6c, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, + 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, + 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3c, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, + 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x42, 0x02, 0x68, 0x00, 0x22, 0xc8, 0x01, 0x0a, 0x20, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x4e, + 0x65, 0x78, 0x75, 0x73, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, + 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x6f, 0x6b, + 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x54, 0x6f, 0x6b, + 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3f, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, + 0x02, 0x68, 0x00, 0x22, 0x23, 0x0a, 0x21, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x4e, 0x65, 0x78, + 0x75, 0x73, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xc3, 0x01, 0x0a, 0x1d, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x64, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, 0x00, 0x12, 0x1e, 0x0a, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, + 0x79, 0x42, 0x02, 0x68, 0x00, 0x12, 0x21, 0x0a, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x6f, 0x6b, + 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x54, 0x6f, 0x6b, + 0x65, 0x6e, 0x42, 0x02, 0x68, 0x00, 0x12, 0x3d, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x6e, 0x65, 0x78, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x61, 0x6e, 0x64, 0x6c, + 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x42, 0x02, 0x68, + 0x00, 0x22, 0x20, 0x0a, 0x1e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x4e, 0x65, 0x78, 0x75, 0x73, + 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0xa5, 0x03, 0x0a, 0x1c, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x4d, 0x75, 0x6c, + 0x74, 0x69, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x20, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x02, 0x68, + 0x00, 0x12, 0x6b, 0x0a, 0x0a, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x4d, 0x75, 0x6c, 0x74, 0x69, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x02, 0x68, 0x00, 0x1a, 0xf5, 0x01, 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x6b, 0x0a, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, + 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0d, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x02, 0x68, 0x00, 0x12, 0x6e, 0x0a, 0x0f, + 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x48, 0x00, 0x52, 0x0e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x42, 0x02, 0x68, 0x00, 0x42, 0x0b, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x22, 0x82, 0x03, 0x0a, 0x1d, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x4d, 0x75, 0x6c, + 0x74, 0x69, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x69, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x4d, 0x75, 0x6c, 0x74, + 0x69, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x09, 0x72, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x73, 0x42, 0x02, 0x68, 0x00, 0x1a, 0xf5, 0x01, 0x0a, 0x08, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x6c, 0x0a, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x0d, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x02, 0x68, 0x00, 0x12, 0x6f, 0x0a, 0x0f, 0x75, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x42, 0x02, 0x68, 0x00, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x42, 0xbe, 0x01, 0x0a, 0x22, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x14, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x35, 0x67, + 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, + 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, + 0x76, 0x31, 0x3b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0xaa, 0x02, 0x21, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, 0x41, 0x70, + 0x69, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x56, 0x31, 0xea, 0x02, 0x24, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x3a, + 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_temporal_api_workflowservice_v1_request_response_proto_rawDescOnce sync.Once + file_temporal_api_workflowservice_v1_request_response_proto_rawDescData = file_temporal_api_workflowservice_v1_request_response_proto_rawDesc +) + +func file_temporal_api_workflowservice_v1_request_response_proto_rawDescGZIP() []byte { + file_temporal_api_workflowservice_v1_request_response_proto_rawDescOnce.Do(func() { + file_temporal_api_workflowservice_v1_request_response_proto_rawDescData = protoimpl.X.CompressGZIP(file_temporal_api_workflowservice_v1_request_response_proto_rawDescData) + }) + return file_temporal_api_workflowservice_v1_request_response_proto_rawDescData +} + +var file_temporal_api_workflowservice_v1_request_response_proto_msgTypes = make([]protoimpl.MessageInfo, 145) +var file_temporal_api_workflowservice_v1_request_response_proto_goTypes = []any{ + (*RegisterNamespaceRequest)(nil), // 0: temporal.api.workflowservice.v1.RegisterNamespaceRequest + (*RegisterNamespaceResponse)(nil), // 1: temporal.api.workflowservice.v1.RegisterNamespaceResponse + (*ListNamespacesRequest)(nil), // 2: temporal.api.workflowservice.v1.ListNamespacesRequest + (*ListNamespacesResponse)(nil), // 3: temporal.api.workflowservice.v1.ListNamespacesResponse + (*DescribeNamespaceRequest)(nil), // 4: temporal.api.workflowservice.v1.DescribeNamespaceRequest + (*DescribeNamespaceResponse)(nil), // 5: temporal.api.workflowservice.v1.DescribeNamespaceResponse + (*UpdateNamespaceRequest)(nil), // 6: temporal.api.workflowservice.v1.UpdateNamespaceRequest + (*UpdateNamespaceResponse)(nil), // 7: temporal.api.workflowservice.v1.UpdateNamespaceResponse + (*DeprecateNamespaceRequest)(nil), // 8: temporal.api.workflowservice.v1.DeprecateNamespaceRequest + (*DeprecateNamespaceResponse)(nil), // 9: temporal.api.workflowservice.v1.DeprecateNamespaceResponse + (*StartWorkflowExecutionRequest)(nil), // 10: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest + (*StartWorkflowExecutionResponse)(nil), // 11: temporal.api.workflowservice.v1.StartWorkflowExecutionResponse + (*GetWorkflowExecutionHistoryRequest)(nil), // 12: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryRequest + (*GetWorkflowExecutionHistoryResponse)(nil), // 13: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryResponse + (*GetWorkflowExecutionHistoryReverseRequest)(nil), // 14: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryReverseRequest + (*GetWorkflowExecutionHistoryReverseResponse)(nil), // 15: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryReverseResponse + (*PollWorkflowTaskQueueRequest)(nil), // 16: temporal.api.workflowservice.v1.PollWorkflowTaskQueueRequest + (*PollWorkflowTaskQueueResponse)(nil), // 17: temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse + (*RespondWorkflowTaskCompletedRequest)(nil), // 18: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest + (*RespondWorkflowTaskCompletedResponse)(nil), // 19: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedResponse + (*RespondWorkflowTaskFailedRequest)(nil), // 20: temporal.api.workflowservice.v1.RespondWorkflowTaskFailedRequest + (*RespondWorkflowTaskFailedResponse)(nil), // 21: temporal.api.workflowservice.v1.RespondWorkflowTaskFailedResponse + (*PollActivityTaskQueueRequest)(nil), // 22: temporal.api.workflowservice.v1.PollActivityTaskQueueRequest + (*PollActivityTaskQueueResponse)(nil), // 23: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse + (*RecordActivityTaskHeartbeatRequest)(nil), // 24: temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatRequest + (*RecordActivityTaskHeartbeatResponse)(nil), // 25: temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatResponse + (*RecordActivityTaskHeartbeatByIdRequest)(nil), // 26: temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatByIdRequest + (*RecordActivityTaskHeartbeatByIdResponse)(nil), // 27: temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatByIdResponse + (*RespondActivityTaskCompletedRequest)(nil), // 28: temporal.api.workflowservice.v1.RespondActivityTaskCompletedRequest + (*RespondActivityTaskCompletedResponse)(nil), // 29: temporal.api.workflowservice.v1.RespondActivityTaskCompletedResponse + (*RespondActivityTaskCompletedByIdRequest)(nil), // 30: temporal.api.workflowservice.v1.RespondActivityTaskCompletedByIdRequest + (*RespondActivityTaskCompletedByIdResponse)(nil), // 31: temporal.api.workflowservice.v1.RespondActivityTaskCompletedByIdResponse + (*RespondActivityTaskFailedRequest)(nil), // 32: temporal.api.workflowservice.v1.RespondActivityTaskFailedRequest + (*RespondActivityTaskFailedResponse)(nil), // 33: temporal.api.workflowservice.v1.RespondActivityTaskFailedResponse + (*RespondActivityTaskFailedByIdRequest)(nil), // 34: temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdRequest + (*RespondActivityTaskFailedByIdResponse)(nil), // 35: temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdResponse + (*RespondActivityTaskCanceledRequest)(nil), // 36: temporal.api.workflowservice.v1.RespondActivityTaskCanceledRequest + (*RespondActivityTaskCanceledResponse)(nil), // 37: temporal.api.workflowservice.v1.RespondActivityTaskCanceledResponse + (*RespondActivityTaskCanceledByIdRequest)(nil), // 38: temporal.api.workflowservice.v1.RespondActivityTaskCanceledByIdRequest + (*RespondActivityTaskCanceledByIdResponse)(nil), // 39: temporal.api.workflowservice.v1.RespondActivityTaskCanceledByIdResponse + (*RequestCancelWorkflowExecutionRequest)(nil), // 40: temporal.api.workflowservice.v1.RequestCancelWorkflowExecutionRequest + (*RequestCancelWorkflowExecutionResponse)(nil), // 41: temporal.api.workflowservice.v1.RequestCancelWorkflowExecutionResponse + (*SignalWorkflowExecutionRequest)(nil), // 42: temporal.api.workflowservice.v1.SignalWorkflowExecutionRequest + (*SignalWorkflowExecutionResponse)(nil), // 43: temporal.api.workflowservice.v1.SignalWorkflowExecutionResponse + (*SignalWithStartWorkflowExecutionRequest)(nil), // 44: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest + (*SignalWithStartWorkflowExecutionResponse)(nil), // 45: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionResponse + (*ResetWorkflowExecutionRequest)(nil), // 46: temporal.api.workflowservice.v1.ResetWorkflowExecutionRequest + (*ResetWorkflowExecutionResponse)(nil), // 47: temporal.api.workflowservice.v1.ResetWorkflowExecutionResponse + (*TerminateWorkflowExecutionRequest)(nil), // 48: temporal.api.workflowservice.v1.TerminateWorkflowExecutionRequest + (*TerminateWorkflowExecutionResponse)(nil), // 49: temporal.api.workflowservice.v1.TerminateWorkflowExecutionResponse + (*DeleteWorkflowExecutionRequest)(nil), // 50: temporal.api.workflowservice.v1.DeleteWorkflowExecutionRequest + (*DeleteWorkflowExecutionResponse)(nil), // 51: temporal.api.workflowservice.v1.DeleteWorkflowExecutionResponse + (*ListOpenWorkflowExecutionsRequest)(nil), // 52: temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsRequest + (*ListOpenWorkflowExecutionsResponse)(nil), // 53: temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsResponse + (*ListClosedWorkflowExecutionsRequest)(nil), // 54: temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsRequest + (*ListClosedWorkflowExecutionsResponse)(nil), // 55: temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsResponse + (*ListWorkflowExecutionsRequest)(nil), // 56: temporal.api.workflowservice.v1.ListWorkflowExecutionsRequest + (*ListWorkflowExecutionsResponse)(nil), // 57: temporal.api.workflowservice.v1.ListWorkflowExecutionsResponse + (*ListArchivedWorkflowExecutionsRequest)(nil), // 58: temporal.api.workflowservice.v1.ListArchivedWorkflowExecutionsRequest + (*ListArchivedWorkflowExecutionsResponse)(nil), // 59: temporal.api.workflowservice.v1.ListArchivedWorkflowExecutionsResponse + (*ScanWorkflowExecutionsRequest)(nil), // 60: temporal.api.workflowservice.v1.ScanWorkflowExecutionsRequest + (*ScanWorkflowExecutionsResponse)(nil), // 61: temporal.api.workflowservice.v1.ScanWorkflowExecutionsResponse + (*CountWorkflowExecutionsRequest)(nil), // 62: temporal.api.workflowservice.v1.CountWorkflowExecutionsRequest + (*CountWorkflowExecutionsResponse)(nil), // 63: temporal.api.workflowservice.v1.CountWorkflowExecutionsResponse + (*GetSearchAttributesRequest)(nil), // 64: temporal.api.workflowservice.v1.GetSearchAttributesRequest + (*GetSearchAttributesResponse)(nil), // 65: temporal.api.workflowservice.v1.GetSearchAttributesResponse + (*RespondQueryTaskCompletedRequest)(nil), // 66: temporal.api.workflowservice.v1.RespondQueryTaskCompletedRequest + (*RespondQueryTaskCompletedResponse)(nil), // 67: temporal.api.workflowservice.v1.RespondQueryTaskCompletedResponse + (*ResetStickyTaskQueueRequest)(nil), // 68: temporal.api.workflowservice.v1.ResetStickyTaskQueueRequest + (*ResetStickyTaskQueueResponse)(nil), // 69: temporal.api.workflowservice.v1.ResetStickyTaskQueueResponse + (*QueryWorkflowRequest)(nil), // 70: temporal.api.workflowservice.v1.QueryWorkflowRequest + (*QueryWorkflowResponse)(nil), // 71: temporal.api.workflowservice.v1.QueryWorkflowResponse + (*DescribeWorkflowExecutionRequest)(nil), // 72: temporal.api.workflowservice.v1.DescribeWorkflowExecutionRequest + (*DescribeWorkflowExecutionResponse)(nil), // 73: temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse + (*DescribeTaskQueueRequest)(nil), // 74: temporal.api.workflowservice.v1.DescribeTaskQueueRequest + (*DescribeTaskQueueResponse)(nil), // 75: temporal.api.workflowservice.v1.DescribeTaskQueueResponse + (*GetClusterInfoRequest)(nil), // 76: temporal.api.workflowservice.v1.GetClusterInfoRequest + (*GetClusterInfoResponse)(nil), // 77: temporal.api.workflowservice.v1.GetClusterInfoResponse + (*GetSystemInfoRequest)(nil), // 78: temporal.api.workflowservice.v1.GetSystemInfoRequest + (*GetSystemInfoResponse)(nil), // 79: temporal.api.workflowservice.v1.GetSystemInfoResponse + (*ListTaskQueuePartitionsRequest)(nil), // 80: temporal.api.workflowservice.v1.ListTaskQueuePartitionsRequest + (*ListTaskQueuePartitionsResponse)(nil), // 81: temporal.api.workflowservice.v1.ListTaskQueuePartitionsResponse + (*CreateScheduleRequest)(nil), // 82: temporal.api.workflowservice.v1.CreateScheduleRequest + (*CreateScheduleResponse)(nil), // 83: temporal.api.workflowservice.v1.CreateScheduleResponse + (*DescribeScheduleRequest)(nil), // 84: temporal.api.workflowservice.v1.DescribeScheduleRequest + (*DescribeScheduleResponse)(nil), // 85: temporal.api.workflowservice.v1.DescribeScheduleResponse + (*UpdateScheduleRequest)(nil), // 86: temporal.api.workflowservice.v1.UpdateScheduleRequest + (*UpdateScheduleResponse)(nil), // 87: temporal.api.workflowservice.v1.UpdateScheduleResponse + (*PatchScheduleRequest)(nil), // 88: temporal.api.workflowservice.v1.PatchScheduleRequest + (*PatchScheduleResponse)(nil), // 89: temporal.api.workflowservice.v1.PatchScheduleResponse + (*ListScheduleMatchingTimesRequest)(nil), // 90: temporal.api.workflowservice.v1.ListScheduleMatchingTimesRequest + (*ListScheduleMatchingTimesResponse)(nil), // 91: temporal.api.workflowservice.v1.ListScheduleMatchingTimesResponse + (*DeleteScheduleRequest)(nil), // 92: temporal.api.workflowservice.v1.DeleteScheduleRequest + (*DeleteScheduleResponse)(nil), // 93: temporal.api.workflowservice.v1.DeleteScheduleResponse + (*ListSchedulesRequest)(nil), // 94: temporal.api.workflowservice.v1.ListSchedulesRequest + (*ListSchedulesResponse)(nil), // 95: temporal.api.workflowservice.v1.ListSchedulesResponse + (*UpdateWorkerBuildIdCompatibilityRequest)(nil), // 96: temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest + (*UpdateWorkerBuildIdCompatibilityResponse)(nil), // 97: temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityResponse + (*GetWorkerBuildIdCompatibilityRequest)(nil), // 98: temporal.api.workflowservice.v1.GetWorkerBuildIdCompatibilityRequest + (*GetWorkerBuildIdCompatibilityResponse)(nil), // 99: temporal.api.workflowservice.v1.GetWorkerBuildIdCompatibilityResponse + (*UpdateWorkerVersioningRulesRequest)(nil), // 100: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest + (*UpdateWorkerVersioningRulesResponse)(nil), // 101: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesResponse + (*GetWorkerVersioningRulesRequest)(nil), // 102: temporal.api.workflowservice.v1.GetWorkerVersioningRulesRequest + (*GetWorkerVersioningRulesResponse)(nil), // 103: temporal.api.workflowservice.v1.GetWorkerVersioningRulesResponse + (*GetWorkerTaskReachabilityRequest)(nil), // 104: temporal.api.workflowservice.v1.GetWorkerTaskReachabilityRequest + (*GetWorkerTaskReachabilityResponse)(nil), // 105: temporal.api.workflowservice.v1.GetWorkerTaskReachabilityResponse + (*UpdateWorkflowExecutionRequest)(nil), // 106: temporal.api.workflowservice.v1.UpdateWorkflowExecutionRequest + (*UpdateWorkflowExecutionResponse)(nil), // 107: temporal.api.workflowservice.v1.UpdateWorkflowExecutionResponse + (*StartBatchOperationRequest)(nil), // 108: temporal.api.workflowservice.v1.StartBatchOperationRequest + (*StartBatchOperationResponse)(nil), // 109: temporal.api.workflowservice.v1.StartBatchOperationResponse + (*StopBatchOperationRequest)(nil), // 110: temporal.api.workflowservice.v1.StopBatchOperationRequest + (*StopBatchOperationResponse)(nil), // 111: temporal.api.workflowservice.v1.StopBatchOperationResponse + (*DescribeBatchOperationRequest)(nil), // 112: temporal.api.workflowservice.v1.DescribeBatchOperationRequest + (*DescribeBatchOperationResponse)(nil), // 113: temporal.api.workflowservice.v1.DescribeBatchOperationResponse + (*ListBatchOperationsRequest)(nil), // 114: temporal.api.workflowservice.v1.ListBatchOperationsRequest + (*ListBatchOperationsResponse)(nil), // 115: temporal.api.workflowservice.v1.ListBatchOperationsResponse + (*PollWorkflowExecutionUpdateRequest)(nil), // 116: temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateRequest + (*PollWorkflowExecutionUpdateResponse)(nil), // 117: temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateResponse + (*PollNexusTaskQueueRequest)(nil), // 118: temporal.api.workflowservice.v1.PollNexusTaskQueueRequest + (*PollNexusTaskQueueResponse)(nil), // 119: temporal.api.workflowservice.v1.PollNexusTaskQueueResponse + (*RespondNexusTaskCompletedRequest)(nil), // 120: temporal.api.workflowservice.v1.RespondNexusTaskCompletedRequest + (*RespondNexusTaskCompletedResponse)(nil), // 121: temporal.api.workflowservice.v1.RespondNexusTaskCompletedResponse + (*RespondNexusTaskFailedRequest)(nil), // 122: temporal.api.workflowservice.v1.RespondNexusTaskFailedRequest + (*RespondNexusTaskFailedResponse)(nil), // 123: temporal.api.workflowservice.v1.RespondNexusTaskFailedResponse + (*ExecuteMultiOperationRequest)(nil), // 124: temporal.api.workflowservice.v1.ExecuteMultiOperationRequest + (*ExecuteMultiOperationResponse)(nil), // 125: temporal.api.workflowservice.v1.ExecuteMultiOperationResponse + nil, // 126: temporal.api.workflowservice.v1.RegisterNamespaceRequest.DataEntry + nil, // 127: temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.QueriesEntry + nil, // 128: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.QueryResultsEntry + (*CountWorkflowExecutionsResponse_AggregationGroup)(nil), // 129: temporal.api.workflowservice.v1.CountWorkflowExecutionsResponse.AggregationGroup + nil, // 130: temporal.api.workflowservice.v1.GetSearchAttributesResponse.KeysEntry + nil, // 131: temporal.api.workflowservice.v1.DescribeTaskQueueResponse.VersionsInfoEntry + nil, // 132: temporal.api.workflowservice.v1.GetClusterInfoResponse.SupportedClientsEntry + (*GetSystemInfoResponse_Capabilities)(nil), // 133: temporal.api.workflowservice.v1.GetSystemInfoResponse.Capabilities + (*UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion)(nil), // 134: temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest.AddNewCompatibleVersion + (*UpdateWorkerBuildIdCompatibilityRequest_MergeSets)(nil), // 135: temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest.MergeSets + (*UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule)(nil), // 136: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.InsertBuildIdAssignmentRule + (*UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule)(nil), // 137: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.ReplaceBuildIdAssignmentRule + (*UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule)(nil), // 138: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.DeleteBuildIdAssignmentRule + (*UpdateWorkerVersioningRulesRequest_AddCompatibleBuildIdRedirectRule)(nil), // 139: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.AddCompatibleBuildIdRedirectRule + (*UpdateWorkerVersioningRulesRequest_ReplaceCompatibleBuildIdRedirectRule)(nil), // 140: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.ReplaceCompatibleBuildIdRedirectRule + (*UpdateWorkerVersioningRulesRequest_DeleteCompatibleBuildIdRedirectRule)(nil), // 141: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.DeleteCompatibleBuildIdRedirectRule + (*UpdateWorkerVersioningRulesRequest_CommitBuildId)(nil), // 142: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.CommitBuildId + (*ExecuteMultiOperationRequest_Operation)(nil), // 143: temporal.api.workflowservice.v1.ExecuteMultiOperationRequest.Operation + (*ExecuteMultiOperationResponse_Response)(nil), // 144: temporal.api.workflowservice.v1.ExecuteMultiOperationResponse.Response + (*durationpb.Duration)(nil), // 145: google.protobuf.Duration + (*v1.ClusterReplicationConfig)(nil), // 146: temporal.api.replication.v1.ClusterReplicationConfig + (v11.ArchivalState)(0), // 147: temporal.api.enums.v1.ArchivalState + (*v12.NamespaceFilter)(nil), // 148: temporal.api.namespace.v1.NamespaceFilter + (*v12.NamespaceInfo)(nil), // 149: temporal.api.namespace.v1.NamespaceInfo + (*v12.NamespaceConfig)(nil), // 150: temporal.api.namespace.v1.NamespaceConfig + (*v1.NamespaceReplicationConfig)(nil), // 151: temporal.api.replication.v1.NamespaceReplicationConfig + (*v1.FailoverStatus)(nil), // 152: temporal.api.replication.v1.FailoverStatus + (*v12.UpdateNamespaceInfo)(nil), // 153: temporal.api.namespace.v1.UpdateNamespaceInfo + (*v13.WorkflowType)(nil), // 154: temporal.api.common.v1.WorkflowType + (*v14.TaskQueue)(nil), // 155: temporal.api.taskqueue.v1.TaskQueue + (*v13.Payloads)(nil), // 156: temporal.api.common.v1.Payloads + (v11.WorkflowIdReusePolicy)(0), // 157: temporal.api.enums.v1.WorkflowIdReusePolicy + (v11.WorkflowIdConflictPolicy)(0), // 158: temporal.api.enums.v1.WorkflowIdConflictPolicy + (*v13.RetryPolicy)(nil), // 159: temporal.api.common.v1.RetryPolicy + (*v13.Memo)(nil), // 160: temporal.api.common.v1.Memo + (*v13.SearchAttributes)(nil), // 161: temporal.api.common.v1.SearchAttributes + (*v13.Header)(nil), // 162: temporal.api.common.v1.Header + (*v15.Failure)(nil), // 163: temporal.api.failure.v1.Failure + (*v13.Callback)(nil), // 164: temporal.api.common.v1.Callback + (*v16.UserMetadata)(nil), // 165: temporal.api.sdk.v1.UserMetadata + (*v13.Link)(nil), // 166: temporal.api.common.v1.Link + (*v13.WorkflowExecution)(nil), // 167: temporal.api.common.v1.WorkflowExecution + (v11.HistoryEventFilterType)(0), // 168: temporal.api.enums.v1.HistoryEventFilterType + (*v17.History)(nil), // 169: temporal.api.history.v1.History + (*v13.DataBlob)(nil), // 170: temporal.api.common.v1.DataBlob + (*v13.WorkerVersionCapabilities)(nil), // 171: temporal.api.common.v1.WorkerVersionCapabilities + (*v18.WorkflowQuery)(nil), // 172: temporal.api.query.v1.WorkflowQuery + (*timestamppb.Timestamp)(nil), // 173: google.protobuf.Timestamp + (*v19.Message)(nil), // 174: temporal.api.protocol.v1.Message + (*v110.Command)(nil), // 175: temporal.api.command.v1.Command + (*v14.StickyExecutionAttributes)(nil), // 176: temporal.api.taskqueue.v1.StickyExecutionAttributes + (*v13.WorkerVersionStamp)(nil), // 177: temporal.api.common.v1.WorkerVersionStamp + (*v16.WorkflowTaskCompletedMetadata)(nil), // 178: temporal.api.sdk.v1.WorkflowTaskCompletedMetadata + (*v13.MeteringMetadata)(nil), // 179: temporal.api.common.v1.MeteringMetadata + (v11.WorkflowTaskFailedCause)(0), // 180: temporal.api.enums.v1.WorkflowTaskFailedCause + (*v14.TaskQueueMetadata)(nil), // 181: temporal.api.taskqueue.v1.TaskQueueMetadata + (*v13.ActivityType)(nil), // 182: temporal.api.common.v1.ActivityType + (v11.ResetReapplyType)(0), // 183: temporal.api.enums.v1.ResetReapplyType + (v11.ResetReapplyExcludeType)(0), // 184: temporal.api.enums.v1.ResetReapplyExcludeType + (*v111.StartTimeFilter)(nil), // 185: temporal.api.filter.v1.StartTimeFilter + (*v111.WorkflowExecutionFilter)(nil), // 186: temporal.api.filter.v1.WorkflowExecutionFilter + (*v111.WorkflowTypeFilter)(nil), // 187: temporal.api.filter.v1.WorkflowTypeFilter + (*v112.WorkflowExecutionInfo)(nil), // 188: temporal.api.workflow.v1.WorkflowExecutionInfo + (*v111.StatusFilter)(nil), // 189: temporal.api.filter.v1.StatusFilter + (v11.QueryResultType)(0), // 190: temporal.api.enums.v1.QueryResultType + (v11.QueryRejectCondition)(0), // 191: temporal.api.enums.v1.QueryRejectCondition + (*v18.QueryRejected)(nil), // 192: temporal.api.query.v1.QueryRejected + (*v112.WorkflowExecutionConfig)(nil), // 193: temporal.api.workflow.v1.WorkflowExecutionConfig + (*v112.PendingActivityInfo)(nil), // 194: temporal.api.workflow.v1.PendingActivityInfo + (*v112.PendingChildExecutionInfo)(nil), // 195: temporal.api.workflow.v1.PendingChildExecutionInfo + (*v112.PendingWorkflowTaskInfo)(nil), // 196: temporal.api.workflow.v1.PendingWorkflowTaskInfo + (*v112.CallbackInfo)(nil), // 197: temporal.api.workflow.v1.CallbackInfo + (*v112.PendingNexusOperationInfo)(nil), // 198: temporal.api.workflow.v1.PendingNexusOperationInfo + (v11.TaskQueueType)(0), // 199: temporal.api.enums.v1.TaskQueueType + (v11.DescribeTaskQueueMode)(0), // 200: temporal.api.enums.v1.DescribeTaskQueueMode + (*v14.TaskQueueVersionSelection)(nil), // 201: temporal.api.taskqueue.v1.TaskQueueVersionSelection + (*v14.PollerInfo)(nil), // 202: temporal.api.taskqueue.v1.PollerInfo + (*v14.TaskQueueStatus)(nil), // 203: temporal.api.taskqueue.v1.TaskQueueStatus + (*v113.VersionInfo)(nil), // 204: temporal.api.version.v1.VersionInfo + (*v14.TaskQueuePartitionMetadata)(nil), // 205: temporal.api.taskqueue.v1.TaskQueuePartitionMetadata + (*v114.Schedule)(nil), // 206: temporal.api.schedule.v1.Schedule + (*v114.SchedulePatch)(nil), // 207: temporal.api.schedule.v1.SchedulePatch + (*v114.ScheduleInfo)(nil), // 208: temporal.api.schedule.v1.ScheduleInfo + (*v114.ScheduleListEntry)(nil), // 209: temporal.api.schedule.v1.ScheduleListEntry + (*v14.CompatibleVersionSet)(nil), // 210: temporal.api.taskqueue.v1.CompatibleVersionSet + (*v14.TimestampedBuildIdAssignmentRule)(nil), // 211: temporal.api.taskqueue.v1.TimestampedBuildIdAssignmentRule + (*v14.TimestampedCompatibleBuildIdRedirectRule)(nil), // 212: temporal.api.taskqueue.v1.TimestampedCompatibleBuildIdRedirectRule + (v11.TaskReachability)(0), // 213: temporal.api.enums.v1.TaskReachability + (*v14.BuildIdReachability)(nil), // 214: temporal.api.taskqueue.v1.BuildIdReachability + (*v115.WaitPolicy)(nil), // 215: temporal.api.update.v1.WaitPolicy + (*v115.Request)(nil), // 216: temporal.api.update.v1.Request + (*v115.UpdateRef)(nil), // 217: temporal.api.update.v1.UpdateRef + (*v115.Outcome)(nil), // 218: temporal.api.update.v1.Outcome + (v11.UpdateWorkflowExecutionLifecycleStage)(0), // 219: temporal.api.enums.v1.UpdateWorkflowExecutionLifecycleStage + (*v116.BatchOperationTermination)(nil), // 220: temporal.api.batch.v1.BatchOperationTermination + (*v116.BatchOperationSignal)(nil), // 221: temporal.api.batch.v1.BatchOperationSignal + (*v116.BatchOperationCancellation)(nil), // 222: temporal.api.batch.v1.BatchOperationCancellation + (*v116.BatchOperationDeletion)(nil), // 223: temporal.api.batch.v1.BatchOperationDeletion + (*v116.BatchOperationReset)(nil), // 224: temporal.api.batch.v1.BatchOperationReset + (v11.BatchOperationType)(0), // 225: temporal.api.enums.v1.BatchOperationType + (v11.BatchOperationState)(0), // 226: temporal.api.enums.v1.BatchOperationState + (*v116.BatchOperationInfo)(nil), // 227: temporal.api.batch.v1.BatchOperationInfo + (*v117.Request)(nil), // 228: temporal.api.nexus.v1.Request + (*v117.Response)(nil), // 229: temporal.api.nexus.v1.Response + (*v117.HandlerError)(nil), // 230: temporal.api.nexus.v1.HandlerError + (*v18.WorkflowQueryResult)(nil), // 231: temporal.api.query.v1.WorkflowQueryResult + (*v13.Payload)(nil), // 232: temporal.api.common.v1.Payload + (v11.IndexedValueType)(0), // 233: temporal.api.enums.v1.IndexedValueType + (*v14.TaskQueueVersionInfo)(nil), // 234: temporal.api.taskqueue.v1.TaskQueueVersionInfo + (*v14.BuildIdAssignmentRule)(nil), // 235: temporal.api.taskqueue.v1.BuildIdAssignmentRule + (*v14.CompatibleBuildIdRedirectRule)(nil), // 236: temporal.api.taskqueue.v1.CompatibleBuildIdRedirectRule +} +var file_temporal_api_workflowservice_v1_request_response_proto_depIdxs = []int32{ + 145, // 0: temporal.api.workflowservice.v1.RegisterNamespaceRequest.workflow_execution_retention_period:type_name -> google.protobuf.Duration + 146, // 1: temporal.api.workflowservice.v1.RegisterNamespaceRequest.clusters:type_name -> temporal.api.replication.v1.ClusterReplicationConfig + 126, // 2: temporal.api.workflowservice.v1.RegisterNamespaceRequest.data:type_name -> temporal.api.workflowservice.v1.RegisterNamespaceRequest.DataEntry + 147, // 3: temporal.api.workflowservice.v1.RegisterNamespaceRequest.history_archival_state:type_name -> temporal.api.enums.v1.ArchivalState + 147, // 4: temporal.api.workflowservice.v1.RegisterNamespaceRequest.visibility_archival_state:type_name -> temporal.api.enums.v1.ArchivalState + 148, // 5: temporal.api.workflowservice.v1.ListNamespacesRequest.namespace_filter:type_name -> temporal.api.namespace.v1.NamespaceFilter + 5, // 6: temporal.api.workflowservice.v1.ListNamespacesResponse.namespaces:type_name -> temporal.api.workflowservice.v1.DescribeNamespaceResponse + 149, // 7: temporal.api.workflowservice.v1.DescribeNamespaceResponse.namespace_info:type_name -> temporal.api.namespace.v1.NamespaceInfo + 150, // 8: temporal.api.workflowservice.v1.DescribeNamespaceResponse.config:type_name -> temporal.api.namespace.v1.NamespaceConfig + 151, // 9: temporal.api.workflowservice.v1.DescribeNamespaceResponse.replication_config:type_name -> temporal.api.replication.v1.NamespaceReplicationConfig + 152, // 10: temporal.api.workflowservice.v1.DescribeNamespaceResponse.failover_history:type_name -> temporal.api.replication.v1.FailoverStatus + 153, // 11: temporal.api.workflowservice.v1.UpdateNamespaceRequest.update_info:type_name -> temporal.api.namespace.v1.UpdateNamespaceInfo + 150, // 12: temporal.api.workflowservice.v1.UpdateNamespaceRequest.config:type_name -> temporal.api.namespace.v1.NamespaceConfig + 151, // 13: temporal.api.workflowservice.v1.UpdateNamespaceRequest.replication_config:type_name -> temporal.api.replication.v1.NamespaceReplicationConfig + 149, // 14: temporal.api.workflowservice.v1.UpdateNamespaceResponse.namespace_info:type_name -> temporal.api.namespace.v1.NamespaceInfo + 150, // 15: temporal.api.workflowservice.v1.UpdateNamespaceResponse.config:type_name -> temporal.api.namespace.v1.NamespaceConfig + 151, // 16: temporal.api.workflowservice.v1.UpdateNamespaceResponse.replication_config:type_name -> temporal.api.replication.v1.NamespaceReplicationConfig + 154, // 17: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 155, // 18: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 156, // 19: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.input:type_name -> temporal.api.common.v1.Payloads + 145, // 20: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.workflow_execution_timeout:type_name -> google.protobuf.Duration + 145, // 21: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.workflow_run_timeout:type_name -> google.protobuf.Duration + 145, // 22: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.workflow_task_timeout:type_name -> google.protobuf.Duration + 157, // 23: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.workflow_id_reuse_policy:type_name -> temporal.api.enums.v1.WorkflowIdReusePolicy + 158, // 24: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.workflow_id_conflict_policy:type_name -> temporal.api.enums.v1.WorkflowIdConflictPolicy + 159, // 25: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.retry_policy:type_name -> temporal.api.common.v1.RetryPolicy + 160, // 26: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.memo:type_name -> temporal.api.common.v1.Memo + 161, // 27: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 162, // 28: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.header:type_name -> temporal.api.common.v1.Header + 163, // 29: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.continued_failure:type_name -> temporal.api.failure.v1.Failure + 156, // 30: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.last_completion_result:type_name -> temporal.api.common.v1.Payloads + 145, // 31: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.workflow_start_delay:type_name -> google.protobuf.Duration + 164, // 32: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.completion_callbacks:type_name -> temporal.api.common.v1.Callback + 165, // 33: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.user_metadata:type_name -> temporal.api.sdk.v1.UserMetadata + 166, // 34: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest.links:type_name -> temporal.api.common.v1.Link + 17, // 35: temporal.api.workflowservice.v1.StartWorkflowExecutionResponse.eager_workflow_task:type_name -> temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse + 167, // 36: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryRequest.execution:type_name -> temporal.api.common.v1.WorkflowExecution + 168, // 37: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryRequest.history_event_filter_type:type_name -> temporal.api.enums.v1.HistoryEventFilterType + 169, // 38: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryResponse.history:type_name -> temporal.api.history.v1.History + 170, // 39: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryResponse.raw_history:type_name -> temporal.api.common.v1.DataBlob + 167, // 40: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryReverseRequest.execution:type_name -> temporal.api.common.v1.WorkflowExecution + 169, // 41: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryReverseResponse.history:type_name -> temporal.api.history.v1.History + 155, // 42: temporal.api.workflowservice.v1.PollWorkflowTaskQueueRequest.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 171, // 43: temporal.api.workflowservice.v1.PollWorkflowTaskQueueRequest.worker_version_capabilities:type_name -> temporal.api.common.v1.WorkerVersionCapabilities + 167, // 44: temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 154, // 45: temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 169, // 46: temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.history:type_name -> temporal.api.history.v1.History + 172, // 47: temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.query:type_name -> temporal.api.query.v1.WorkflowQuery + 155, // 48: temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.workflow_execution_task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 173, // 49: temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.scheduled_time:type_name -> google.protobuf.Timestamp + 173, // 50: temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.started_time:type_name -> google.protobuf.Timestamp + 127, // 51: temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.queries:type_name -> temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.QueriesEntry + 174, // 52: temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.messages:type_name -> temporal.api.protocol.v1.Message + 175, // 53: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.commands:type_name -> temporal.api.command.v1.Command + 176, // 54: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.sticky_attributes:type_name -> temporal.api.taskqueue.v1.StickyExecutionAttributes + 128, // 55: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.query_results:type_name -> temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.QueryResultsEntry + 177, // 56: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.worker_version_stamp:type_name -> temporal.api.common.v1.WorkerVersionStamp + 174, // 57: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.messages:type_name -> temporal.api.protocol.v1.Message + 178, // 58: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.sdk_metadata:type_name -> temporal.api.sdk.v1.WorkflowTaskCompletedMetadata + 179, // 59: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.metering_metadata:type_name -> temporal.api.common.v1.MeteringMetadata + 17, // 60: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedResponse.workflow_task:type_name -> temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse + 23, // 61: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedResponse.activity_tasks:type_name -> temporal.api.workflowservice.v1.PollActivityTaskQueueResponse + 180, // 62: temporal.api.workflowservice.v1.RespondWorkflowTaskFailedRequest.cause:type_name -> temporal.api.enums.v1.WorkflowTaskFailedCause + 163, // 63: temporal.api.workflowservice.v1.RespondWorkflowTaskFailedRequest.failure:type_name -> temporal.api.failure.v1.Failure + 174, // 64: temporal.api.workflowservice.v1.RespondWorkflowTaskFailedRequest.messages:type_name -> temporal.api.protocol.v1.Message + 177, // 65: temporal.api.workflowservice.v1.RespondWorkflowTaskFailedRequest.worker_version:type_name -> temporal.api.common.v1.WorkerVersionStamp + 155, // 66: temporal.api.workflowservice.v1.PollActivityTaskQueueRequest.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 181, // 67: temporal.api.workflowservice.v1.PollActivityTaskQueueRequest.task_queue_metadata:type_name -> temporal.api.taskqueue.v1.TaskQueueMetadata + 171, // 68: temporal.api.workflowservice.v1.PollActivityTaskQueueRequest.worker_version_capabilities:type_name -> temporal.api.common.v1.WorkerVersionCapabilities + 154, // 69: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 167, // 70: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 182, // 71: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse.activity_type:type_name -> temporal.api.common.v1.ActivityType + 162, // 72: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse.header:type_name -> temporal.api.common.v1.Header + 156, // 73: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse.input:type_name -> temporal.api.common.v1.Payloads + 156, // 74: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse.heartbeat_details:type_name -> temporal.api.common.v1.Payloads + 173, // 75: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse.scheduled_time:type_name -> google.protobuf.Timestamp + 173, // 76: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse.current_attempt_scheduled_time:type_name -> google.protobuf.Timestamp + 173, // 77: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse.started_time:type_name -> google.protobuf.Timestamp + 145, // 78: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse.schedule_to_close_timeout:type_name -> google.protobuf.Duration + 145, // 79: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse.start_to_close_timeout:type_name -> google.protobuf.Duration + 145, // 80: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse.heartbeat_timeout:type_name -> google.protobuf.Duration + 159, // 81: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse.retry_policy:type_name -> temporal.api.common.v1.RetryPolicy + 156, // 82: temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatRequest.details:type_name -> temporal.api.common.v1.Payloads + 156, // 83: temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatByIdRequest.details:type_name -> temporal.api.common.v1.Payloads + 156, // 84: temporal.api.workflowservice.v1.RespondActivityTaskCompletedRequest.result:type_name -> temporal.api.common.v1.Payloads + 177, // 85: temporal.api.workflowservice.v1.RespondActivityTaskCompletedRequest.worker_version:type_name -> temporal.api.common.v1.WorkerVersionStamp + 156, // 86: temporal.api.workflowservice.v1.RespondActivityTaskCompletedByIdRequest.result:type_name -> temporal.api.common.v1.Payloads + 163, // 87: temporal.api.workflowservice.v1.RespondActivityTaskFailedRequest.failure:type_name -> temporal.api.failure.v1.Failure + 156, // 88: temporal.api.workflowservice.v1.RespondActivityTaskFailedRequest.last_heartbeat_details:type_name -> temporal.api.common.v1.Payloads + 177, // 89: temporal.api.workflowservice.v1.RespondActivityTaskFailedRequest.worker_version:type_name -> temporal.api.common.v1.WorkerVersionStamp + 163, // 90: temporal.api.workflowservice.v1.RespondActivityTaskFailedResponse.failures:type_name -> temporal.api.failure.v1.Failure + 163, // 91: temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdRequest.failure:type_name -> temporal.api.failure.v1.Failure + 156, // 92: temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdRequest.last_heartbeat_details:type_name -> temporal.api.common.v1.Payloads + 163, // 93: temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdResponse.failures:type_name -> temporal.api.failure.v1.Failure + 156, // 94: temporal.api.workflowservice.v1.RespondActivityTaskCanceledRequest.details:type_name -> temporal.api.common.v1.Payloads + 177, // 95: temporal.api.workflowservice.v1.RespondActivityTaskCanceledRequest.worker_version:type_name -> temporal.api.common.v1.WorkerVersionStamp + 156, // 96: temporal.api.workflowservice.v1.RespondActivityTaskCanceledByIdRequest.details:type_name -> temporal.api.common.v1.Payloads + 167, // 97: temporal.api.workflowservice.v1.RequestCancelWorkflowExecutionRequest.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 167, // 98: temporal.api.workflowservice.v1.SignalWorkflowExecutionRequest.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 156, // 99: temporal.api.workflowservice.v1.SignalWorkflowExecutionRequest.input:type_name -> temporal.api.common.v1.Payloads + 162, // 100: temporal.api.workflowservice.v1.SignalWorkflowExecutionRequest.header:type_name -> temporal.api.common.v1.Header + 154, // 101: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.workflow_type:type_name -> temporal.api.common.v1.WorkflowType + 155, // 102: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 156, // 103: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.input:type_name -> temporal.api.common.v1.Payloads + 145, // 104: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.workflow_execution_timeout:type_name -> google.protobuf.Duration + 145, // 105: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.workflow_run_timeout:type_name -> google.protobuf.Duration + 145, // 106: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.workflow_task_timeout:type_name -> google.protobuf.Duration + 157, // 107: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.workflow_id_reuse_policy:type_name -> temporal.api.enums.v1.WorkflowIdReusePolicy + 158, // 108: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.workflow_id_conflict_policy:type_name -> temporal.api.enums.v1.WorkflowIdConflictPolicy + 156, // 109: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.signal_input:type_name -> temporal.api.common.v1.Payloads + 159, // 110: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.retry_policy:type_name -> temporal.api.common.v1.RetryPolicy + 160, // 111: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.memo:type_name -> temporal.api.common.v1.Memo + 161, // 112: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 162, // 113: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.header:type_name -> temporal.api.common.v1.Header + 145, // 114: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.workflow_start_delay:type_name -> google.protobuf.Duration + 165, // 115: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest.user_metadata:type_name -> temporal.api.sdk.v1.UserMetadata + 167, // 116: temporal.api.workflowservice.v1.ResetWorkflowExecutionRequest.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 183, // 117: temporal.api.workflowservice.v1.ResetWorkflowExecutionRequest.reset_reapply_type:type_name -> temporal.api.enums.v1.ResetReapplyType + 184, // 118: temporal.api.workflowservice.v1.ResetWorkflowExecutionRequest.reset_reapply_exclude_types:type_name -> temporal.api.enums.v1.ResetReapplyExcludeType + 167, // 119: temporal.api.workflowservice.v1.TerminateWorkflowExecutionRequest.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 156, // 120: temporal.api.workflowservice.v1.TerminateWorkflowExecutionRequest.details:type_name -> temporal.api.common.v1.Payloads + 167, // 121: temporal.api.workflowservice.v1.DeleteWorkflowExecutionRequest.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 185, // 122: temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsRequest.start_time_filter:type_name -> temporal.api.filter.v1.StartTimeFilter + 186, // 123: temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsRequest.execution_filter:type_name -> temporal.api.filter.v1.WorkflowExecutionFilter + 187, // 124: temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsRequest.type_filter:type_name -> temporal.api.filter.v1.WorkflowTypeFilter + 188, // 125: temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsResponse.executions:type_name -> temporal.api.workflow.v1.WorkflowExecutionInfo + 185, // 126: temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsRequest.start_time_filter:type_name -> temporal.api.filter.v1.StartTimeFilter + 186, // 127: temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsRequest.execution_filter:type_name -> temporal.api.filter.v1.WorkflowExecutionFilter + 187, // 128: temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsRequest.type_filter:type_name -> temporal.api.filter.v1.WorkflowTypeFilter + 189, // 129: temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsRequest.status_filter:type_name -> temporal.api.filter.v1.StatusFilter + 188, // 130: temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsResponse.executions:type_name -> temporal.api.workflow.v1.WorkflowExecutionInfo + 188, // 131: temporal.api.workflowservice.v1.ListWorkflowExecutionsResponse.executions:type_name -> temporal.api.workflow.v1.WorkflowExecutionInfo + 188, // 132: temporal.api.workflowservice.v1.ListArchivedWorkflowExecutionsResponse.executions:type_name -> temporal.api.workflow.v1.WorkflowExecutionInfo + 188, // 133: temporal.api.workflowservice.v1.ScanWorkflowExecutionsResponse.executions:type_name -> temporal.api.workflow.v1.WorkflowExecutionInfo + 129, // 134: temporal.api.workflowservice.v1.CountWorkflowExecutionsResponse.groups:type_name -> temporal.api.workflowservice.v1.CountWorkflowExecutionsResponse.AggregationGroup + 130, // 135: temporal.api.workflowservice.v1.GetSearchAttributesResponse.keys:type_name -> temporal.api.workflowservice.v1.GetSearchAttributesResponse.KeysEntry + 190, // 136: temporal.api.workflowservice.v1.RespondQueryTaskCompletedRequest.completed_type:type_name -> temporal.api.enums.v1.QueryResultType + 156, // 137: temporal.api.workflowservice.v1.RespondQueryTaskCompletedRequest.query_result:type_name -> temporal.api.common.v1.Payloads + 167, // 138: temporal.api.workflowservice.v1.ResetStickyTaskQueueRequest.execution:type_name -> temporal.api.common.v1.WorkflowExecution + 167, // 139: temporal.api.workflowservice.v1.QueryWorkflowRequest.execution:type_name -> temporal.api.common.v1.WorkflowExecution + 172, // 140: temporal.api.workflowservice.v1.QueryWorkflowRequest.query:type_name -> temporal.api.query.v1.WorkflowQuery + 191, // 141: temporal.api.workflowservice.v1.QueryWorkflowRequest.query_reject_condition:type_name -> temporal.api.enums.v1.QueryRejectCondition + 156, // 142: temporal.api.workflowservice.v1.QueryWorkflowResponse.query_result:type_name -> temporal.api.common.v1.Payloads + 192, // 143: temporal.api.workflowservice.v1.QueryWorkflowResponse.query_rejected:type_name -> temporal.api.query.v1.QueryRejected + 167, // 144: temporal.api.workflowservice.v1.DescribeWorkflowExecutionRequest.execution:type_name -> temporal.api.common.v1.WorkflowExecution + 193, // 145: temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse.execution_config:type_name -> temporal.api.workflow.v1.WorkflowExecutionConfig + 188, // 146: temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse.workflow_execution_info:type_name -> temporal.api.workflow.v1.WorkflowExecutionInfo + 194, // 147: temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse.pending_activities:type_name -> temporal.api.workflow.v1.PendingActivityInfo + 195, // 148: temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse.pending_children:type_name -> temporal.api.workflow.v1.PendingChildExecutionInfo + 196, // 149: temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse.pending_workflow_task:type_name -> temporal.api.workflow.v1.PendingWorkflowTaskInfo + 197, // 150: temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse.callbacks:type_name -> temporal.api.workflow.v1.CallbackInfo + 198, // 151: temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse.pending_nexus_operations:type_name -> temporal.api.workflow.v1.PendingNexusOperationInfo + 155, // 152: temporal.api.workflowservice.v1.DescribeTaskQueueRequest.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 199, // 153: temporal.api.workflowservice.v1.DescribeTaskQueueRequest.task_queue_type:type_name -> temporal.api.enums.v1.TaskQueueType + 200, // 154: temporal.api.workflowservice.v1.DescribeTaskQueueRequest.api_mode:type_name -> temporal.api.enums.v1.DescribeTaskQueueMode + 201, // 155: temporal.api.workflowservice.v1.DescribeTaskQueueRequest.versions:type_name -> temporal.api.taskqueue.v1.TaskQueueVersionSelection + 199, // 156: temporal.api.workflowservice.v1.DescribeTaskQueueRequest.task_queue_types:type_name -> temporal.api.enums.v1.TaskQueueType + 202, // 157: temporal.api.workflowservice.v1.DescribeTaskQueueResponse.pollers:type_name -> temporal.api.taskqueue.v1.PollerInfo + 203, // 158: temporal.api.workflowservice.v1.DescribeTaskQueueResponse.task_queue_status:type_name -> temporal.api.taskqueue.v1.TaskQueueStatus + 131, // 159: temporal.api.workflowservice.v1.DescribeTaskQueueResponse.versions_info:type_name -> temporal.api.workflowservice.v1.DescribeTaskQueueResponse.VersionsInfoEntry + 132, // 160: temporal.api.workflowservice.v1.GetClusterInfoResponse.supported_clients:type_name -> temporal.api.workflowservice.v1.GetClusterInfoResponse.SupportedClientsEntry + 204, // 161: temporal.api.workflowservice.v1.GetClusterInfoResponse.version_info:type_name -> temporal.api.version.v1.VersionInfo + 133, // 162: temporal.api.workflowservice.v1.GetSystemInfoResponse.capabilities:type_name -> temporal.api.workflowservice.v1.GetSystemInfoResponse.Capabilities + 155, // 163: temporal.api.workflowservice.v1.ListTaskQueuePartitionsRequest.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 205, // 164: temporal.api.workflowservice.v1.ListTaskQueuePartitionsResponse.activity_task_queue_partitions:type_name -> temporal.api.taskqueue.v1.TaskQueuePartitionMetadata + 205, // 165: temporal.api.workflowservice.v1.ListTaskQueuePartitionsResponse.workflow_task_queue_partitions:type_name -> temporal.api.taskqueue.v1.TaskQueuePartitionMetadata + 206, // 166: temporal.api.workflowservice.v1.CreateScheduleRequest.schedule:type_name -> temporal.api.schedule.v1.Schedule + 207, // 167: temporal.api.workflowservice.v1.CreateScheduleRequest.initial_patch:type_name -> temporal.api.schedule.v1.SchedulePatch + 160, // 168: temporal.api.workflowservice.v1.CreateScheduleRequest.memo:type_name -> temporal.api.common.v1.Memo + 161, // 169: temporal.api.workflowservice.v1.CreateScheduleRequest.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 206, // 170: temporal.api.workflowservice.v1.DescribeScheduleResponse.schedule:type_name -> temporal.api.schedule.v1.Schedule + 208, // 171: temporal.api.workflowservice.v1.DescribeScheduleResponse.info:type_name -> temporal.api.schedule.v1.ScheduleInfo + 160, // 172: temporal.api.workflowservice.v1.DescribeScheduleResponse.memo:type_name -> temporal.api.common.v1.Memo + 161, // 173: temporal.api.workflowservice.v1.DescribeScheduleResponse.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 206, // 174: temporal.api.workflowservice.v1.UpdateScheduleRequest.schedule:type_name -> temporal.api.schedule.v1.Schedule + 161, // 175: temporal.api.workflowservice.v1.UpdateScheduleRequest.search_attributes:type_name -> temporal.api.common.v1.SearchAttributes + 207, // 176: temporal.api.workflowservice.v1.PatchScheduleRequest.patch:type_name -> temporal.api.schedule.v1.SchedulePatch + 173, // 177: temporal.api.workflowservice.v1.ListScheduleMatchingTimesRequest.start_time:type_name -> google.protobuf.Timestamp + 173, // 178: temporal.api.workflowservice.v1.ListScheduleMatchingTimesRequest.end_time:type_name -> google.protobuf.Timestamp + 173, // 179: temporal.api.workflowservice.v1.ListScheduleMatchingTimesResponse.start_time:type_name -> google.protobuf.Timestamp + 209, // 180: temporal.api.workflowservice.v1.ListSchedulesResponse.schedules:type_name -> temporal.api.schedule.v1.ScheduleListEntry + 134, // 181: temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest.add_new_compatible_build_id:type_name -> temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest.AddNewCompatibleVersion + 135, // 182: temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest.merge_sets:type_name -> temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest.MergeSets + 210, // 183: temporal.api.workflowservice.v1.GetWorkerBuildIdCompatibilityResponse.major_version_sets:type_name -> temporal.api.taskqueue.v1.CompatibleVersionSet + 136, // 184: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.insert_assignment_rule:type_name -> temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.InsertBuildIdAssignmentRule + 137, // 185: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.replace_assignment_rule:type_name -> temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.ReplaceBuildIdAssignmentRule + 138, // 186: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.delete_assignment_rule:type_name -> temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.DeleteBuildIdAssignmentRule + 139, // 187: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.add_compatible_redirect_rule:type_name -> temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.AddCompatibleBuildIdRedirectRule + 140, // 188: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.replace_compatible_redirect_rule:type_name -> temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.ReplaceCompatibleBuildIdRedirectRule + 141, // 189: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.delete_compatible_redirect_rule:type_name -> temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.DeleteCompatibleBuildIdRedirectRule + 142, // 190: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.commit_build_id:type_name -> temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.CommitBuildId + 211, // 191: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesResponse.assignment_rules:type_name -> temporal.api.taskqueue.v1.TimestampedBuildIdAssignmentRule + 212, // 192: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesResponse.compatible_redirect_rules:type_name -> temporal.api.taskqueue.v1.TimestampedCompatibleBuildIdRedirectRule + 211, // 193: temporal.api.workflowservice.v1.GetWorkerVersioningRulesResponse.assignment_rules:type_name -> temporal.api.taskqueue.v1.TimestampedBuildIdAssignmentRule + 212, // 194: temporal.api.workflowservice.v1.GetWorkerVersioningRulesResponse.compatible_redirect_rules:type_name -> temporal.api.taskqueue.v1.TimestampedCompatibleBuildIdRedirectRule + 213, // 195: temporal.api.workflowservice.v1.GetWorkerTaskReachabilityRequest.reachability:type_name -> temporal.api.enums.v1.TaskReachability + 214, // 196: temporal.api.workflowservice.v1.GetWorkerTaskReachabilityResponse.build_id_reachability:type_name -> temporal.api.taskqueue.v1.BuildIdReachability + 167, // 197: temporal.api.workflowservice.v1.UpdateWorkflowExecutionRequest.workflow_execution:type_name -> temporal.api.common.v1.WorkflowExecution + 215, // 198: temporal.api.workflowservice.v1.UpdateWorkflowExecutionRequest.wait_policy:type_name -> temporal.api.update.v1.WaitPolicy + 216, // 199: temporal.api.workflowservice.v1.UpdateWorkflowExecutionRequest.request:type_name -> temporal.api.update.v1.Request + 217, // 200: temporal.api.workflowservice.v1.UpdateWorkflowExecutionResponse.update_ref:type_name -> temporal.api.update.v1.UpdateRef + 218, // 201: temporal.api.workflowservice.v1.UpdateWorkflowExecutionResponse.outcome:type_name -> temporal.api.update.v1.Outcome + 219, // 202: temporal.api.workflowservice.v1.UpdateWorkflowExecutionResponse.stage:type_name -> temporal.api.enums.v1.UpdateWorkflowExecutionLifecycleStage + 167, // 203: temporal.api.workflowservice.v1.StartBatchOperationRequest.executions:type_name -> temporal.api.common.v1.WorkflowExecution + 220, // 204: temporal.api.workflowservice.v1.StartBatchOperationRequest.termination_operation:type_name -> temporal.api.batch.v1.BatchOperationTermination + 221, // 205: temporal.api.workflowservice.v1.StartBatchOperationRequest.signal_operation:type_name -> temporal.api.batch.v1.BatchOperationSignal + 222, // 206: temporal.api.workflowservice.v1.StartBatchOperationRequest.cancellation_operation:type_name -> temporal.api.batch.v1.BatchOperationCancellation + 223, // 207: temporal.api.workflowservice.v1.StartBatchOperationRequest.deletion_operation:type_name -> temporal.api.batch.v1.BatchOperationDeletion + 224, // 208: temporal.api.workflowservice.v1.StartBatchOperationRequest.reset_operation:type_name -> temporal.api.batch.v1.BatchOperationReset + 225, // 209: temporal.api.workflowservice.v1.DescribeBatchOperationResponse.operation_type:type_name -> temporal.api.enums.v1.BatchOperationType + 226, // 210: temporal.api.workflowservice.v1.DescribeBatchOperationResponse.state:type_name -> temporal.api.enums.v1.BatchOperationState + 173, // 211: temporal.api.workflowservice.v1.DescribeBatchOperationResponse.start_time:type_name -> google.protobuf.Timestamp + 173, // 212: temporal.api.workflowservice.v1.DescribeBatchOperationResponse.close_time:type_name -> google.protobuf.Timestamp + 227, // 213: temporal.api.workflowservice.v1.ListBatchOperationsResponse.operation_info:type_name -> temporal.api.batch.v1.BatchOperationInfo + 217, // 214: temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateRequest.update_ref:type_name -> temporal.api.update.v1.UpdateRef + 215, // 215: temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateRequest.wait_policy:type_name -> temporal.api.update.v1.WaitPolicy + 218, // 216: temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateResponse.outcome:type_name -> temporal.api.update.v1.Outcome + 219, // 217: temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateResponse.stage:type_name -> temporal.api.enums.v1.UpdateWorkflowExecutionLifecycleStage + 217, // 218: temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateResponse.update_ref:type_name -> temporal.api.update.v1.UpdateRef + 155, // 219: temporal.api.workflowservice.v1.PollNexusTaskQueueRequest.task_queue:type_name -> temporal.api.taskqueue.v1.TaskQueue + 171, // 220: temporal.api.workflowservice.v1.PollNexusTaskQueueRequest.worker_version_capabilities:type_name -> temporal.api.common.v1.WorkerVersionCapabilities + 228, // 221: temporal.api.workflowservice.v1.PollNexusTaskQueueResponse.request:type_name -> temporal.api.nexus.v1.Request + 229, // 222: temporal.api.workflowservice.v1.RespondNexusTaskCompletedRequest.response:type_name -> temporal.api.nexus.v1.Response + 230, // 223: temporal.api.workflowservice.v1.RespondNexusTaskFailedRequest.error:type_name -> temporal.api.nexus.v1.HandlerError + 143, // 224: temporal.api.workflowservice.v1.ExecuteMultiOperationRequest.operations:type_name -> temporal.api.workflowservice.v1.ExecuteMultiOperationRequest.Operation + 144, // 225: temporal.api.workflowservice.v1.ExecuteMultiOperationResponse.responses:type_name -> temporal.api.workflowservice.v1.ExecuteMultiOperationResponse.Response + 172, // 226: temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse.QueriesEntry.value:type_name -> temporal.api.query.v1.WorkflowQuery + 231, // 227: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest.QueryResultsEntry.value:type_name -> temporal.api.query.v1.WorkflowQueryResult + 232, // 228: temporal.api.workflowservice.v1.CountWorkflowExecutionsResponse.AggregationGroup.group_values:type_name -> temporal.api.common.v1.Payload + 233, // 229: temporal.api.workflowservice.v1.GetSearchAttributesResponse.KeysEntry.value:type_name -> temporal.api.enums.v1.IndexedValueType + 234, // 230: temporal.api.workflowservice.v1.DescribeTaskQueueResponse.VersionsInfoEntry.value:type_name -> temporal.api.taskqueue.v1.TaskQueueVersionInfo + 235, // 231: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.InsertBuildIdAssignmentRule.rule:type_name -> temporal.api.taskqueue.v1.BuildIdAssignmentRule + 235, // 232: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.ReplaceBuildIdAssignmentRule.rule:type_name -> temporal.api.taskqueue.v1.BuildIdAssignmentRule + 236, // 233: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.AddCompatibleBuildIdRedirectRule.rule:type_name -> temporal.api.taskqueue.v1.CompatibleBuildIdRedirectRule + 236, // 234: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest.ReplaceCompatibleBuildIdRedirectRule.rule:type_name -> temporal.api.taskqueue.v1.CompatibleBuildIdRedirectRule + 10, // 235: temporal.api.workflowservice.v1.ExecuteMultiOperationRequest.Operation.start_workflow:type_name -> temporal.api.workflowservice.v1.StartWorkflowExecutionRequest + 106, // 236: temporal.api.workflowservice.v1.ExecuteMultiOperationRequest.Operation.update_workflow:type_name -> temporal.api.workflowservice.v1.UpdateWorkflowExecutionRequest + 11, // 237: temporal.api.workflowservice.v1.ExecuteMultiOperationResponse.Response.start_workflow:type_name -> temporal.api.workflowservice.v1.StartWorkflowExecutionResponse + 107, // 238: temporal.api.workflowservice.v1.ExecuteMultiOperationResponse.Response.update_workflow:type_name -> temporal.api.workflowservice.v1.UpdateWorkflowExecutionResponse + 239, // [239:239] is the sub-list for method output_type + 239, // [239:239] is the sub-list for method input_type + 239, // [239:239] is the sub-list for extension type_name + 239, // [239:239] is the sub-list for extension extendee + 0, // [0:239] is the sub-list for field type_name +} + +func init() { file_temporal_api_workflowservice_v1_request_response_proto_init() } +func file_temporal_api_workflowservice_v1_request_response_proto_init() { + if File_temporal_api_workflowservice_v1_request_response_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*RegisterNamespaceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*RegisterNamespaceResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[2].Exporter = func(v any, i int) any { + switch v := v.(*ListNamespacesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[3].Exporter = func(v any, i int) any { + switch v := v.(*ListNamespacesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[4].Exporter = func(v any, i int) any { + switch v := v.(*DescribeNamespaceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[5].Exporter = func(v any, i int) any { + switch v := v.(*DescribeNamespaceResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[6].Exporter = func(v any, i int) any { + switch v := v.(*UpdateNamespaceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[7].Exporter = func(v any, i int) any { + switch v := v.(*UpdateNamespaceResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[8].Exporter = func(v any, i int) any { + switch v := v.(*DeprecateNamespaceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[9].Exporter = func(v any, i int) any { + switch v := v.(*DeprecateNamespaceResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[10].Exporter = func(v any, i int) any { + switch v := v.(*StartWorkflowExecutionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[11].Exporter = func(v any, i int) any { + switch v := v.(*StartWorkflowExecutionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[12].Exporter = func(v any, i int) any { + switch v := v.(*GetWorkflowExecutionHistoryRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[13].Exporter = func(v any, i int) any { + switch v := v.(*GetWorkflowExecutionHistoryResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[14].Exporter = func(v any, i int) any { + switch v := v.(*GetWorkflowExecutionHistoryReverseRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[15].Exporter = func(v any, i int) any { + switch v := v.(*GetWorkflowExecutionHistoryReverseResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[16].Exporter = func(v any, i int) any { + switch v := v.(*PollWorkflowTaskQueueRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[17].Exporter = func(v any, i int) any { + switch v := v.(*PollWorkflowTaskQueueResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[18].Exporter = func(v any, i int) any { + switch v := v.(*RespondWorkflowTaskCompletedRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[19].Exporter = func(v any, i int) any { + switch v := v.(*RespondWorkflowTaskCompletedResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[20].Exporter = func(v any, i int) any { + switch v := v.(*RespondWorkflowTaskFailedRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[21].Exporter = func(v any, i int) any { + switch v := v.(*RespondWorkflowTaskFailedResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[22].Exporter = func(v any, i int) any { + switch v := v.(*PollActivityTaskQueueRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[23].Exporter = func(v any, i int) any { + switch v := v.(*PollActivityTaskQueueResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[24].Exporter = func(v any, i int) any { + switch v := v.(*RecordActivityTaskHeartbeatRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[25].Exporter = func(v any, i int) any { + switch v := v.(*RecordActivityTaskHeartbeatResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[26].Exporter = func(v any, i int) any { + switch v := v.(*RecordActivityTaskHeartbeatByIdRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[27].Exporter = func(v any, i int) any { + switch v := v.(*RecordActivityTaskHeartbeatByIdResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[28].Exporter = func(v any, i int) any { + switch v := v.(*RespondActivityTaskCompletedRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[29].Exporter = func(v any, i int) any { + switch v := v.(*RespondActivityTaskCompletedResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[30].Exporter = func(v any, i int) any { + switch v := v.(*RespondActivityTaskCompletedByIdRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[31].Exporter = func(v any, i int) any { + switch v := v.(*RespondActivityTaskCompletedByIdResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[32].Exporter = func(v any, i int) any { + switch v := v.(*RespondActivityTaskFailedRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[33].Exporter = func(v any, i int) any { + switch v := v.(*RespondActivityTaskFailedResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[34].Exporter = func(v any, i int) any { + switch v := v.(*RespondActivityTaskFailedByIdRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[35].Exporter = func(v any, i int) any { + switch v := v.(*RespondActivityTaskFailedByIdResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[36].Exporter = func(v any, i int) any { + switch v := v.(*RespondActivityTaskCanceledRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[37].Exporter = func(v any, i int) any { + switch v := v.(*RespondActivityTaskCanceledResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[38].Exporter = func(v any, i int) any { + switch v := v.(*RespondActivityTaskCanceledByIdRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[39].Exporter = func(v any, i int) any { + switch v := v.(*RespondActivityTaskCanceledByIdResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[40].Exporter = func(v any, i int) any { + switch v := v.(*RequestCancelWorkflowExecutionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[41].Exporter = func(v any, i int) any { + switch v := v.(*RequestCancelWorkflowExecutionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[42].Exporter = func(v any, i int) any { + switch v := v.(*SignalWorkflowExecutionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[43].Exporter = func(v any, i int) any { + switch v := v.(*SignalWorkflowExecutionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[44].Exporter = func(v any, i int) any { + switch v := v.(*SignalWithStartWorkflowExecutionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[45].Exporter = func(v any, i int) any { + switch v := v.(*SignalWithStartWorkflowExecutionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[46].Exporter = func(v any, i int) any { + switch v := v.(*ResetWorkflowExecutionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[47].Exporter = func(v any, i int) any { + switch v := v.(*ResetWorkflowExecutionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[48].Exporter = func(v any, i int) any { + switch v := v.(*TerminateWorkflowExecutionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[49].Exporter = func(v any, i int) any { + switch v := v.(*TerminateWorkflowExecutionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[50].Exporter = func(v any, i int) any { + switch v := v.(*DeleteWorkflowExecutionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[51].Exporter = func(v any, i int) any { + switch v := v.(*DeleteWorkflowExecutionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[52].Exporter = func(v any, i int) any { + switch v := v.(*ListOpenWorkflowExecutionsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[53].Exporter = func(v any, i int) any { + switch v := v.(*ListOpenWorkflowExecutionsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[54].Exporter = func(v any, i int) any { + switch v := v.(*ListClosedWorkflowExecutionsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[55].Exporter = func(v any, i int) any { + switch v := v.(*ListClosedWorkflowExecutionsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[56].Exporter = func(v any, i int) any { + switch v := v.(*ListWorkflowExecutionsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[57].Exporter = func(v any, i int) any { + switch v := v.(*ListWorkflowExecutionsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[58].Exporter = func(v any, i int) any { + switch v := v.(*ListArchivedWorkflowExecutionsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[59].Exporter = func(v any, i int) any { + switch v := v.(*ListArchivedWorkflowExecutionsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[60].Exporter = func(v any, i int) any { + switch v := v.(*ScanWorkflowExecutionsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[61].Exporter = func(v any, i int) any { + switch v := v.(*ScanWorkflowExecutionsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[62].Exporter = func(v any, i int) any { + switch v := v.(*CountWorkflowExecutionsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[63].Exporter = func(v any, i int) any { + switch v := v.(*CountWorkflowExecutionsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[64].Exporter = func(v any, i int) any { + switch v := v.(*GetSearchAttributesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[65].Exporter = func(v any, i int) any { + switch v := v.(*GetSearchAttributesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[66].Exporter = func(v any, i int) any { + switch v := v.(*RespondQueryTaskCompletedRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[67].Exporter = func(v any, i int) any { + switch v := v.(*RespondQueryTaskCompletedResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[68].Exporter = func(v any, i int) any { + switch v := v.(*ResetStickyTaskQueueRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[69].Exporter = func(v any, i int) any { + switch v := v.(*ResetStickyTaskQueueResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[70].Exporter = func(v any, i int) any { + switch v := v.(*QueryWorkflowRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[71].Exporter = func(v any, i int) any { + switch v := v.(*QueryWorkflowResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[72].Exporter = func(v any, i int) any { + switch v := v.(*DescribeWorkflowExecutionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[73].Exporter = func(v any, i int) any { + switch v := v.(*DescribeWorkflowExecutionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[74].Exporter = func(v any, i int) any { + switch v := v.(*DescribeTaskQueueRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[75].Exporter = func(v any, i int) any { + switch v := v.(*DescribeTaskQueueResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[76].Exporter = func(v any, i int) any { + switch v := v.(*GetClusterInfoRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[77].Exporter = func(v any, i int) any { + switch v := v.(*GetClusterInfoResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[78].Exporter = func(v any, i int) any { + switch v := v.(*GetSystemInfoRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[79].Exporter = func(v any, i int) any { + switch v := v.(*GetSystemInfoResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[80].Exporter = func(v any, i int) any { + switch v := v.(*ListTaskQueuePartitionsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[81].Exporter = func(v any, i int) any { + switch v := v.(*ListTaskQueuePartitionsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[82].Exporter = func(v any, i int) any { + switch v := v.(*CreateScheduleRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[83].Exporter = func(v any, i int) any { + switch v := v.(*CreateScheduleResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[84].Exporter = func(v any, i int) any { + switch v := v.(*DescribeScheduleRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[85].Exporter = func(v any, i int) any { + switch v := v.(*DescribeScheduleResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[86].Exporter = func(v any, i int) any { + switch v := v.(*UpdateScheduleRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[87].Exporter = func(v any, i int) any { + switch v := v.(*UpdateScheduleResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[88].Exporter = func(v any, i int) any { + switch v := v.(*PatchScheduleRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[89].Exporter = func(v any, i int) any { + switch v := v.(*PatchScheduleResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[90].Exporter = func(v any, i int) any { + switch v := v.(*ListScheduleMatchingTimesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[91].Exporter = func(v any, i int) any { + switch v := v.(*ListScheduleMatchingTimesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[92].Exporter = func(v any, i int) any { + switch v := v.(*DeleteScheduleRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[93].Exporter = func(v any, i int) any { + switch v := v.(*DeleteScheduleResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[94].Exporter = func(v any, i int) any { + switch v := v.(*ListSchedulesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[95].Exporter = func(v any, i int) any { + switch v := v.(*ListSchedulesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[96].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkerBuildIdCompatibilityRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[97].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkerBuildIdCompatibilityResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[98].Exporter = func(v any, i int) any { + switch v := v.(*GetWorkerBuildIdCompatibilityRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[99].Exporter = func(v any, i int) any { + switch v := v.(*GetWorkerBuildIdCompatibilityResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[100].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkerVersioningRulesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[101].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkerVersioningRulesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[102].Exporter = func(v any, i int) any { + switch v := v.(*GetWorkerVersioningRulesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[103].Exporter = func(v any, i int) any { + switch v := v.(*GetWorkerVersioningRulesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[104].Exporter = func(v any, i int) any { + switch v := v.(*GetWorkerTaskReachabilityRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[105].Exporter = func(v any, i int) any { + switch v := v.(*GetWorkerTaskReachabilityResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[106].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkflowExecutionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[107].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkflowExecutionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[108].Exporter = func(v any, i int) any { + switch v := v.(*StartBatchOperationRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[109].Exporter = func(v any, i int) any { + switch v := v.(*StartBatchOperationResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[110].Exporter = func(v any, i int) any { + switch v := v.(*StopBatchOperationRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[111].Exporter = func(v any, i int) any { + switch v := v.(*StopBatchOperationResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[112].Exporter = func(v any, i int) any { + switch v := v.(*DescribeBatchOperationRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[113].Exporter = func(v any, i int) any { + switch v := v.(*DescribeBatchOperationResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[114].Exporter = func(v any, i int) any { + switch v := v.(*ListBatchOperationsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[115].Exporter = func(v any, i int) any { + switch v := v.(*ListBatchOperationsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[116].Exporter = func(v any, i int) any { + switch v := v.(*PollWorkflowExecutionUpdateRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[117].Exporter = func(v any, i int) any { + switch v := v.(*PollWorkflowExecutionUpdateResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[118].Exporter = func(v any, i int) any { + switch v := v.(*PollNexusTaskQueueRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[119].Exporter = func(v any, i int) any { + switch v := v.(*PollNexusTaskQueueResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[120].Exporter = func(v any, i int) any { + switch v := v.(*RespondNexusTaskCompletedRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[121].Exporter = func(v any, i int) any { + switch v := v.(*RespondNexusTaskCompletedResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[122].Exporter = func(v any, i int) any { + switch v := v.(*RespondNexusTaskFailedRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[123].Exporter = func(v any, i int) any { + switch v := v.(*RespondNexusTaskFailedResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[124].Exporter = func(v any, i int) any { + switch v := v.(*ExecuteMultiOperationRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[125].Exporter = func(v any, i int) any { + switch v := v.(*ExecuteMultiOperationResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[129].Exporter = func(v any, i int) any { + switch v := v.(*CountWorkflowExecutionsResponse_AggregationGroup); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[133].Exporter = func(v any, i int) any { + switch v := v.(*GetSystemInfoResponse_Capabilities); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[134].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleVersion); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[135].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkerBuildIdCompatibilityRequest_MergeSets); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[136].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkerVersioningRulesRequest_InsertBuildIdAssignmentRule); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[137].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkerVersioningRulesRequest_ReplaceBuildIdAssignmentRule); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[138].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkerVersioningRulesRequest_DeleteBuildIdAssignmentRule); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[139].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkerVersioningRulesRequest_AddCompatibleBuildIdRedirectRule); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[140].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkerVersioningRulesRequest_ReplaceCompatibleBuildIdRedirectRule); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[141].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkerVersioningRulesRequest_DeleteCompatibleBuildIdRedirectRule); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[142].Exporter = func(v any, i int) any { + switch v := v.(*UpdateWorkerVersioningRulesRequest_CommitBuildId); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[143].Exporter = func(v any, i int) any { + switch v := v.(*ExecuteMultiOperationRequest_Operation); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[144].Exporter = func(v any, i int) any { + switch v := v.(*ExecuteMultiOperationResponse_Response); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[52].OneofWrappers = []any{ + (*ListOpenWorkflowExecutionsRequest_ExecutionFilter)(nil), + (*ListOpenWorkflowExecutionsRequest_TypeFilter)(nil), + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[54].OneofWrappers = []any{ + (*ListClosedWorkflowExecutionsRequest_ExecutionFilter)(nil), + (*ListClosedWorkflowExecutionsRequest_TypeFilter)(nil), + (*ListClosedWorkflowExecutionsRequest_StatusFilter)(nil), + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[96].OneofWrappers = []any{ + (*UpdateWorkerBuildIdCompatibilityRequest_AddNewBuildIdInNewDefaultSet)(nil), + (*UpdateWorkerBuildIdCompatibilityRequest_AddNewCompatibleBuildId)(nil), + (*UpdateWorkerBuildIdCompatibilityRequest_PromoteSetByBuildId)(nil), + (*UpdateWorkerBuildIdCompatibilityRequest_PromoteBuildIdWithinSet)(nil), + (*UpdateWorkerBuildIdCompatibilityRequest_MergeSets_)(nil), + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[100].OneofWrappers = []any{ + (*UpdateWorkerVersioningRulesRequest_InsertAssignmentRule)(nil), + (*UpdateWorkerVersioningRulesRequest_ReplaceAssignmentRule)(nil), + (*UpdateWorkerVersioningRulesRequest_DeleteAssignmentRule)(nil), + (*UpdateWorkerVersioningRulesRequest_AddCompatibleRedirectRule)(nil), + (*UpdateWorkerVersioningRulesRequest_ReplaceCompatibleRedirectRule)(nil), + (*UpdateWorkerVersioningRulesRequest_DeleteCompatibleRedirectRule)(nil), + (*UpdateWorkerVersioningRulesRequest_CommitBuildId_)(nil), + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[108].OneofWrappers = []any{ + (*StartBatchOperationRequest_TerminationOperation)(nil), + (*StartBatchOperationRequest_SignalOperation)(nil), + (*StartBatchOperationRequest_CancellationOperation)(nil), + (*StartBatchOperationRequest_DeletionOperation)(nil), + (*StartBatchOperationRequest_ResetOperation)(nil), + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[143].OneofWrappers = []any{ + (*ExecuteMultiOperationRequest_Operation_StartWorkflow)(nil), + (*ExecuteMultiOperationRequest_Operation_UpdateWorkflow)(nil), + } + file_temporal_api_workflowservice_v1_request_response_proto_msgTypes[144].OneofWrappers = []any{ + (*ExecuteMultiOperationResponse_Response_StartWorkflow)(nil), + (*ExecuteMultiOperationResponse_Response_UpdateWorkflow)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_workflowservice_v1_request_response_proto_rawDesc, + NumEnums: 0, + NumMessages: 145, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_temporal_api_workflowservice_v1_request_response_proto_goTypes, + DependencyIndexes: file_temporal_api_workflowservice_v1_request_response_proto_depIdxs, + MessageInfos: file_temporal_api_workflowservice_v1_request_response_proto_msgTypes, + }.Build() + File_temporal_api_workflowservice_v1_request_response_proto = out.File + file_temporal_api_workflowservice_v1_request_response_proto_rawDesc = nil + file_temporal_api_workflowservice_v1_request_response_proto_goTypes = nil + file_temporal_api_workflowservice_v1_request_response_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/workflowservice/v1/service.pb.go b/vendor/go.temporal.io/api/workflowservice/v1/service.pb.go new file mode 100644 index 00000000000..29f3ef12fa8 --- /dev/null +++ b/vendor/go.temporal.io/api/workflowservice/v1/service.pb.go @@ -0,0 +1,1332 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// plugins: +// protoc-gen-go +// protoc +// source: temporal/api/workflowservice/v1/service.proto + +package workflowservice + +import ( + reflect "reflect" + + _ "google.golang.org/genproto/googleapis/api/annotations" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +var File_temporal_api_workflowservice_v1_service_proto protoreflect.FileDescriptor + +var file_temporal_api_workflowservice_v1_service_proto_rawDesc = []byte{ + 0x0a, 0x2d, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, 0x76, + 0x31, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, + 0x1f, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, + 0x1a, 0x36, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, 0x76, + 0x31, 0x2f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x32, 0xcb, 0x79, 0x0a, 0x0f, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0xc3, 0x01, 0x0a, 0x11, 0x52, + 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x12, 0x39, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, + 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x37, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x31, 0x3a, + 0x01, 0x2a, 0x5a, 0x17, 0x3a, 0x01, 0x2a, 0x22, 0x12, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, + 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x22, 0x13, 0x2f, 0x63, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, + 0x12, 0xd5, 0x01, 0x0a, 0x11, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x39, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x3a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x49, 0x82, + 0xd3, 0xe4, 0x93, 0x02, 0x43, 0x5a, 0x20, 0x12, 0x1e, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, + 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x12, 0x1f, 0x2f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x12, 0xb4, 0x01, 0x0a, 0x0e, 0x4c, 0x69, 0x73, + 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x12, 0x36, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x37, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x31, 0x82, 0xd3, + 0xe4, 0x93, 0x02, 0x2b, 0x5a, 0x14, 0x12, 0x12, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x12, 0x13, 0x2f, 0x63, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x12, + 0xe3, 0x01, 0x0a, 0x0f, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x12, 0x37, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x38, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x5d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x57, 0x3a, 0x01, + 0x2a, 0x5a, 0x2a, 0x3a, 0x01, 0x2a, 0x22, 0x25, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x22, 0x26, 0x2f, + 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x75, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x8f, 0x01, 0x0a, 0x12, 0x44, 0x65, 0x70, 0x72, 0x65, 0x63, + 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x3a, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, + 0x65, 0x70, 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x70, 0x72, 0x65, + 0x63, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x92, 0x02, 0x0a, 0x16, 0x53, 0x74, 0x61, 0x72, + 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x77, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x71, 0x3a, 0x01, 0x2a, 0x5a, 0x3b, + 0x3a, 0x01, 0x2a, 0x22, 0x36, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x7d, 0x22, 0x2f, 0x2f, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0xa5, 0x02, 0x0a, + 0x15, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, + 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x4d, + 0x75, 0x6c, 0x74, 0x69, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x8c, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x85, 0x01, 0x3a, + 0x01, 0x2a, 0x5a, 0x45, 0x3a, 0x01, 0x2a, 0x22, 0x40, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, + 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x73, 0x2f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x2d, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x2d, + 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x39, 0x2f, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x65, 0x2d, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x2d, 0x6f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0xc1, 0x02, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x69, 0x73, + 0x74, 0x6f, 0x72, 0x79, 0x12, 0x43, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x69, 0x73, 0x74, 0x6f, + 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x44, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x96, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x8f, 0x01, 0x5a, 0x4a, 0x12, 0x48, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, + 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x68, 0x69, + 0x73, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x41, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x7d, + 0x2f, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x12, 0xe6, 0x02, 0x0a, 0x22, 0x47, 0x65, 0x74, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x12, + 0x4a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x76, + 0x65, 0x72, 0x73, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x4b, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, + 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa6, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0x9f, 0x01, 0x5a, 0x52, 0x12, 0x50, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2d, 0x72, + 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x12, 0x49, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, + 0x7d, 0x2f, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x2d, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, + 0x65, 0x12, 0x98, 0x01, 0x0a, 0x15, 0x50, 0x6f, 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x12, 0x3d, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6f, + 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, + 0x65, 0x75, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3e, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6f, 0x6c, + 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, + 0x75, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0xad, 0x01, 0x0a, + 0x1c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x12, 0x44, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, + 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x45, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0xa4, 0x01, 0x0a, + 0x19, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x12, 0x41, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x73, 0x6b, + 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x42, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, + 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x98, 0x01, 0x0a, 0x15, 0x50, 0x6f, 0x6c, 0x6c, 0x41, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x12, 0x3d, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x50, 0x6f, 0x6c, 0x6c, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, + 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3e, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, + 0x6f, 0x6c, 0x6c, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x51, + 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x9b, + 0x02, 0x0a, 0x1b, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, + 0x79, 0x54, 0x61, 0x73, 0x6b, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0x43, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, + 0x61, 0x73, 0x6b, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x44, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x41, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, + 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x71, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0x6b, 0x3a, 0x01, 0x2a, 0x5a, 0x38, 0x3a, 0x01, 0x2a, 0x22, 0x33, 0x2f, 0x61, 0x70, 0x69, 0x2f, + 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, + 0x74, 0x69, 0x65, 0x73, 0x2f, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x22, 0x2c, + 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, + 0x65, 0x73, 0x2f, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0xb3, 0x02, 0x0a, + 0x1f, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, + 0x61, 0x73, 0x6b, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x42, 0x79, 0x49, 0x64, + 0x12, 0x47, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, + 0x79, 0x54, 0x61, 0x73, 0x6b, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x42, 0x79, + 0x49, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x48, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x63, 0x6f, + 0x72, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x48, 0x65, + 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x42, 0x79, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x77, 0x3a, 0x01, 0x2a, 0x5a, 0x3e, + 0x3a, 0x01, 0x2a, 0x22, 0x39, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, 0x65, 0x73, 0x2f, 0x68, + 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x2d, 0x62, 0x79, 0x2d, 0x69, 0x64, 0x22, 0x32, + 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, + 0x65, 0x73, 0x2f, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x2d, 0x62, 0x79, 0x2d, + 0x69, 0x64, 0x12, 0x9c, 0x02, 0x0a, 0x1c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, + 0x74, 0x65, 0x64, 0x12, 0x44, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, + 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x45, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, + 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x6f, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x69, 0x3a, 0x01, 0x2a, 0x5a, 0x37, 0x3a, 0x01, 0x2a, + 0x22, 0x32, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, + 0x2f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, 0x65, 0x73, 0x2f, 0x63, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x22, 0x2b, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, 0x65, 0x73, 0x2f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x12, 0xb4, 0x02, 0x0a, 0x20, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, + 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x42, 0x79, 0x49, 0x64, 0x12, 0x48, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, + 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x64, 0x42, 0x79, 0x49, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x49, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, + 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x42, + 0x79, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x7b, 0x82, 0xd3, 0xe4, + 0x93, 0x02, 0x75, 0x3a, 0x01, 0x2a, 0x5a, 0x3d, 0x3a, 0x01, 0x2a, 0x22, 0x38, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, + 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x69, 0x65, 0x73, 0x2f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x2d, + 0x62, 0x79, 0x2d, 0x69, 0x64, 0x22, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, + 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, 0x65, 0x73, 0x2f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, + 0x74, 0x65, 0x2d, 0x62, 0x79, 0x2d, 0x69, 0x64, 0x12, 0x8b, 0x02, 0x0a, 0x19, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, + 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x12, 0x41, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, + 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, + 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x42, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x46, + 0x61, 0x69, 0x6c, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x67, 0x82, + 0xd3, 0xe4, 0x93, 0x02, 0x61, 0x3a, 0x01, 0x2a, 0x5a, 0x33, 0x3a, 0x01, 0x2a, 0x22, 0x2e, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x63, + 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, 0x65, 0x73, 0x2f, 0x66, 0x61, 0x69, 0x6c, 0x22, 0x27, 0x2f, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, 0x65, + 0x73, 0x2f, 0x66, 0x61, 0x69, 0x6c, 0x12, 0xa3, 0x02, 0x0a, 0x1d, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, + 0x69, 0x6c, 0x65, 0x64, 0x42, 0x79, 0x49, 0x64, 0x12, 0x45, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, + 0x69, 0x6c, 0x65, 0x64, 0x42, 0x79, 0x49, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x46, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, + 0x79, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x42, 0x79, 0x49, 0x64, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x73, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x6d, 0x3a, + 0x01, 0x2a, 0x5a, 0x39, 0x3a, 0x01, 0x2a, 0x22, 0x34, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, + 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, + 0x65, 0x73, 0x2f, 0x66, 0x61, 0x69, 0x6c, 0x2d, 0x62, 0x79, 0x2d, 0x69, 0x64, 0x22, 0x2d, 0x2f, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, 0x65, + 0x73, 0x2f, 0x66, 0x61, 0x69, 0x6c, 0x2d, 0x62, 0x79, 0x2d, 0x69, 0x64, 0x12, 0x95, 0x02, 0x0a, + 0x1b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, + 0x54, 0x61, 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x12, 0x43, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, + 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x44, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, + 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x6b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x65, 0x3a, + 0x01, 0x2a, 0x5a, 0x35, 0x3a, 0x01, 0x2a, 0x22, 0x30, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, + 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, + 0x65, 0x73, 0x2f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x22, 0x29, 0x2f, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x7d, 0x2f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x69, 0x65, 0x73, 0x2f, 0x63, 0x61, + 0x6e, 0x63, 0x65, 0x6c, 0x12, 0xad, 0x02, 0x0a, 0x1f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, + 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, + 0x65, 0x6c, 0x65, 0x64, 0x42, 0x79, 0x49, 0x64, 0x12, 0x47, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x61, + 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x42, 0x79, 0x49, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x48, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x76, + 0x69, 0x74, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x42, + 0x79, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x77, 0x82, 0xd3, 0xe4, + 0x93, 0x02, 0x71, 0x3a, 0x01, 0x2a, 0x5a, 0x3b, 0x3a, 0x01, 0x2a, 0x22, 0x36, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, + 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x63, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x69, 0x65, 0x73, 0x2f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x2d, 0x62, 0x79, + 0x2d, 0x69, 0x64, 0x22, 0x2f, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, + 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x63, 0x74, + 0x69, 0x76, 0x69, 0x74, 0x69, 0x65, 0x73, 0x2f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x2d, 0x62, + 0x79, 0x2d, 0x69, 0x64, 0x12, 0xe0, 0x02, 0x0a, 0x1e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x46, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x47, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xac, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0xa5, 0x01, 0x3a, 0x01, 0x2a, 0x5a, 0x55, 0x3a, 0x01, 0x2a, 0x22, 0x50, 0x2f, 0x61, 0x70, 0x69, + 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x22, 0x49, 0x2f, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, + 0x7b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x7d, + 0x2f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x12, 0xe7, 0x02, 0x0a, 0x17, 0x53, 0x69, 0x67, 0x6e, + 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xc8, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0xc1, 0x01, + 0x3a, 0x01, 0x2a, 0x5a, 0x63, 0x3a, 0x01, 0x2a, 0x22, 0x5e, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, + 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x2f, 0x7b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x69, 0x64, 0x7d, 0x2f, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x2f, 0x7b, 0x73, 0x69, 0x67, 0x6e, + 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x22, 0x57, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x73, 0x69, 0x67, + 0x6e, 0x61, 0x6c, 0x2f, 0x7b, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x7d, 0x12, 0xf2, 0x02, 0x0a, 0x20, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x57, 0x69, 0x74, 0x68, + 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x48, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x57, + 0x69, 0x74, 0x68, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x49, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x57, 0x69, 0x74, 0x68, 0x53, 0x74, 0x61, + 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xb8, 0x01, 0x82, 0xd3, + 0xe4, 0x93, 0x02, 0xb1, 0x01, 0x3a, 0x01, 0x2a, 0x5a, 0x5b, 0x3a, 0x01, 0x2a, 0x22, 0x56, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x2d, 0x77, 0x69, 0x74, + 0x68, 0x2d, 0x73, 0x74, 0x61, 0x72, 0x74, 0x2f, 0x7b, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x22, 0x4f, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x2d, 0x77, 0x69, + 0x74, 0x68, 0x2d, 0x73, 0x74, 0x61, 0x72, 0x74, 0x2f, 0x7b, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x12, 0xc6, 0x02, 0x0a, 0x16, 0x52, 0x65, 0x73, 0x65, 0x74, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0xaa, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0xa3, 0x01, 0x3a, 0x01, 0x2a, 0x5a, + 0x54, 0x3a, 0x01, 0x2a, 0x22, 0x4f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x7d, 0x2f, + 0x72, 0x65, 0x73, 0x65, 0x74, 0x22, 0x48, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x72, 0x65, 0x73, 0x65, 0x74, 0x12, + 0xda, 0x02, 0x0a, 0x1a, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x42, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x43, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xb2, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0xab, + 0x01, 0x3a, 0x01, 0x2a, 0x5a, 0x58, 0x3a, 0x01, 0x2a, 0x22, 0x53, 0x2f, 0x61, 0x70, 0x69, 0x2f, + 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x73, 0x2f, 0x7b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x22, 0x4c, + 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x73, 0x2f, 0x7b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, + 0x64, 0x7d, 0x2f, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x12, 0x9e, 0x01, 0x0a, + 0x17, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0xa7, 0x01, + 0x0a, 0x1a, 0x4c, 0x69, 0x73, 0x74, 0x4f, 0x70, 0x65, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x42, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, + 0x69, 0x73, 0x74, 0x4f, 0x70, 0x65, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x43, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4f, 0x70, 0x65, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0xad, 0x01, 0x0a, 0x1c, 0x4c, 0x69, 0x73, 0x74, + 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x44, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, + 0x6c, 0x6f, 0x73, 0x65, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x45, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0xf0, 0x01, 0x0a, 0x16, 0x4c, 0x69, 0x73, 0x74, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x55, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x4f, 0x5a, 0x2a, 0x12, 0x28, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x21, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, + 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x9a, 0x02, 0x0a, 0x1e, 0x4c, + 0x69, 0x73, 0x74, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x46, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, 0x64, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x47, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x72, 0x63, 0x68, + 0x69, 0x76, 0x65, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x67, + 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x61, 0x5a, 0x33, 0x12, 0x31, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, + 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, + 0x64, 0x2d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x2a, 0x2f, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x61, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, 0x64, 0x2d, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x9b, 0x01, 0x0a, 0x16, 0x53, 0x63, 0x61, 0x6e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x61, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x61, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0xfd, 0x01, 0x0a, 0x17, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x12, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x5f, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x59, 0x5a, 0x2f, 0x12, 0x2d, + 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2d, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x26, 0x2f, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2d, + 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x92, 0x01, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x53, 0x65, 0x61, + 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x3b, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x47, 0x65, 0x74, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3c, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, + 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0xa4, 0x01, 0x0a, 0x19, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x51, 0x75, 0x65, 0x72, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, + 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x12, 0x41, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x64, 0x51, 0x75, 0x65, 0x72, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, + 0x65, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x42, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x51, 0x75, 0x65, 0x72, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, + 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x95, 0x01, 0x0a, 0x14, 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x74, 0x69, 0x63, 0x6b, + 0x79, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x12, 0x3c, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, + 0x65, 0x74, 0x53, 0x74, 0x69, 0x63, 0x6b, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3d, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, + 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, + 0x53, 0x74, 0x69, 0x63, 0x6b, 0x79, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0xbf, 0x02, 0x0a, 0x0d, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x35, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xbe, 0x01, 0x82, 0xd3, 0xe4, + 0x93, 0x02, 0xb7, 0x01, 0x3a, 0x01, 0x2a, 0x5a, 0x5e, 0x3a, 0x01, 0x2a, 0x22, 0x59, 0x2f, 0x61, + 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, + 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x2f, 0x7b, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x7d, 0x22, 0x52, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, + 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, + 0x64, 0x7d, 0x2f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2f, 0x7b, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x7d, 0x12, 0xaa, 0x02, 0x0a, 0x19, + 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x41, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x62, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x42, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x85, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x7f, 0x5a, 0x42, 0x12, 0x40, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, + 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x2f, 0x7b, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0x39, 0x2f, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, + 0x2f, 0x7b, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0x89, 0x02, 0x0a, 0x11, 0x44, 0x65, 0x73, + 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x12, 0x39, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, + 0x75, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x62, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x77, 0x5a, 0x3e, 0x12, + 0x3c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, + 0x74, 0x61, 0x73, 0x6b, 0x2d, 0x71, 0x75, 0x65, 0x75, 0x65, 0x73, 0x2f, 0x7b, 0x74, 0x61, 0x73, + 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x12, 0x35, 0x2f, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x74, 0x61, 0x73, 0x6b, 0x2d, 0x71, 0x75, 0x65, 0x75, + 0x65, 0x73, 0x2f, 0x7b, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x2e, 0x6e, + 0x61, 0x6d, 0x65, 0x7d, 0x12, 0xab, 0x01, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x43, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x36, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x37, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x28, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x22, + 0x5a, 0x16, 0x12, 0x14, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x2d, 0x69, 0x6e, 0x66, 0x6f, 0x12, 0x08, 0x2f, 0x63, 0x6c, 0x75, 0x73, 0x74, + 0x65, 0x72, 0x12, 0xab, 0x01, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x74, 0x65, + 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, + 0x74, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x2b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x25, 0x5a, 0x15, 0x12, 0x13, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x2d, 0x69, 0x6e, + 0x66, 0x6f, 0x12, 0x0c, 0x2f, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x2d, 0x69, 0x6e, 0x66, 0x6f, + 0x12, 0x9e, 0x01, 0x0a, 0x17, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, + 0x75, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x3f, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, + 0x69, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0xfa, 0x01, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x12, 0x36, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x37, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x77, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x71, 0x3a, 0x01, 0x2a, + 0x5a, 0x3b, 0x3a, 0x01, 0x2a, 0x22, 0x36, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x2f, + 0x7b, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x22, 0x2f, 0x2f, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, + 0x2f, 0x7b, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0xfa, + 0x01, 0x0a, 0x10, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x53, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x12, 0x38, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x53, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x39, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x71, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x6b, + 0x5a, 0x38, 0x12, 0x36, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x7d, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x2f, 0x7b, 0x73, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0x2f, 0x2f, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x7d, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x2f, 0x7b, 0x73, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0x89, 0x02, 0x0a, 0x0e, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x12, 0x36, + 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x37, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, + 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x85, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x7f, 0x3a, 0x01, 0x2a, 0x5a, 0x42, 0x3a, 0x01, 0x2a, + 0x22, 0x3d, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, + 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x2f, 0x7b, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x22, + 0x36, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x73, 0x2f, 0x7b, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x7d, + 0x2f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x84, 0x02, 0x0a, 0x0d, 0x50, 0x61, 0x74, 0x63, + 0x68, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x12, 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x74, 0x63, + 0x68, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x36, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x61, 0x74, 0x63, 0x68, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x83, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0x7d, 0x3a, 0x01, 0x2a, 0x5a, 0x41, 0x3a, 0x01, 0x2a, 0x22, 0x3c, 0x2f, 0x61, 0x70, 0x69, 0x2f, + 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, + 0x6c, 0x65, 0x73, 0x2f, 0x7b, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, + 0x7d, 0x2f, 0x70, 0x61, 0x74, 0x63, 0x68, 0x22, 0x35, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, + 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x2f, 0x7b, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x70, 0x61, 0x74, 0x63, 0x68, 0x12, 0xb5, + 0x02, 0x0a, 0x19, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4d, + 0x61, 0x74, 0x63, 0x68, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x12, 0x41, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, + 0x69, 0x73, 0x74, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4d, 0x61, 0x74, 0x63, 0x68, + 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x42, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4d, 0x61, + 0x74, 0x63, 0x68, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x90, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x89, 0x01, 0x5a, 0x47, 0x12, + 0x45, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x2f, 0x7b, 0x73, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x69, 0x6e, 0x67, + 0x2d, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x12, 0x3e, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x2f, 0x7b, 0x73, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x69, 0x6e, 0x67, + 0x2d, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x12, 0xf4, 0x01, 0x0a, 0x0e, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x12, 0x36, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x37, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, + 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x71, 0x82, 0xd3, 0xe4, 0x93, + 0x02, 0x6b, 0x5a, 0x38, 0x2a, 0x36, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x2f, 0x7b, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x2a, 0x2f, 0x2f, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x2f, + 0x7b, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0xd5, 0x01, + 0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x12, + 0x35, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x55, + 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x4f, 0x5a, 0x2a, 0x12, 0x28, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, + 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, + 0x65, 0x73, 0x12, 0x21, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, + 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x73, 0x12, 0xb9, 0x01, 0x0a, 0x20, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x43, 0x6f, 0x6d, + 0x70, 0x61, 0x74, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x12, 0x48, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x49, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, + 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0xe1, 0x02, 0x0a, 0x1d, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x42, + 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x69, 0x6c, + 0x69, 0x74, 0x79, 0x12, 0x45, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x42, + 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x69, 0x6c, + 0x69, 0x74, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x46, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, + 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x64, 0x43, 0x6f, 0x6d, + 0x70, 0x61, 0x74, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0xb0, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0xa9, 0x01, 0x5a, 0x57, 0x12, 0x55, + 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x74, + 0x61, 0x73, 0x6b, 0x2d, 0x71, 0x75, 0x65, 0x75, 0x65, 0x73, 0x2f, 0x7b, 0x74, 0x61, 0x73, 0x6b, + 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x2d, 0x62, + 0x75, 0x69, 0x6c, 0x64, 0x2d, 0x69, 0x64, 0x2d, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, + 0x69, 0x6c, 0x69, 0x74, 0x79, 0x12, 0x4e, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x74, + 0x61, 0x73, 0x6b, 0x2d, 0x71, 0x75, 0x65, 0x75, 0x65, 0x73, 0x2f, 0x7b, 0x74, 0x61, 0x73, 0x6b, + 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x2d, 0x62, + 0x75, 0x69, 0x6c, 0x64, 0x2d, 0x69, 0x64, 0x2d, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, + 0x69, 0x6c, 0x69, 0x74, 0x79, 0x12, 0xaa, 0x01, 0x0a, 0x1b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, + 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x43, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x52, 0x75, + 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x44, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0xc6, 0x02, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, + 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, + 0x40, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x41, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa4, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x9d, 0x01, 0x5a, 0x51, + 0x12, 0x4f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, + 0x2f, 0x74, 0x61, 0x73, 0x6b, 0x2d, 0x71, 0x75, 0x65, 0x75, 0x65, 0x73, 0x2f, 0x7b, 0x74, 0x61, + 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, + 0x2d, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x2d, 0x72, 0x75, 0x6c, 0x65, + 0x73, 0x12, 0x48, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x74, 0x61, 0x73, 0x6b, 0x2d, + 0x71, 0x75, 0x65, 0x75, 0x65, 0x73, 0x2f, 0x7b, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x71, 0x75, 0x65, + 0x75, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x2d, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x2d, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x97, 0x02, 0x0a, 0x19, + 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x61, + 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x12, 0x41, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x57, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, + 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x42, 0x2e, 0x74, + 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x47, + 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x61, 0x63, + 0x68, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x73, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x6d, 0x5a, 0x39, 0x12, 0x37, 0x2f, 0x61, 0x70, 0x69, + 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x65, + 0x72, 0x2d, 0x74, 0x61, 0x73, 0x6b, 0x2d, 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x69, 0x6c, + 0x69, 0x74, 0x79, 0x12, 0x30, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, + 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x2d, 0x74, 0x61, 0x73, 0x6b, 0x2d, 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, + 0x69, 0x6c, 0x69, 0x74, 0x79, 0x12, 0xf5, 0x02, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xd6, 0x01, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0xcf, 0x01, 0x3a, 0x01, + 0x2a, 0x5a, 0x6a, 0x3a, 0x01, 0x2a, 0x22, 0x65, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, + 0x2f, 0x7b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, + 0x7d, 0x2f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2f, 0x7b, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x2e, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x22, 0x5e, 0x2f, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, + 0x2f, 0x7b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, + 0x7d, 0x2f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2f, 0x7b, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x2e, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x12, 0xaa, 0x01, + 0x0a, 0x1b, 0x50, 0x6f, 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x43, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x50, 0x6f, 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x44, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6f, 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8d, 0x02, 0x0a, 0x13, 0x53, + 0x74, 0x61, 0x72, 0x74, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x3c, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x7b, 0x82, + 0xd3, 0xe4, 0x93, 0x02, 0x75, 0x3a, 0x01, 0x2a, 0x5a, 0x3d, 0x3a, 0x01, 0x2a, 0x22, 0x38, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x62, 0x61, + 0x74, 0x63, 0x68, 0x2d, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x7b, + 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x7d, 0x22, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, + 0x2f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x2d, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x2f, 0x7b, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0x95, 0x02, 0x0a, 0x12, 0x53, + 0x74, 0x6f, 0x70, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x3a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3b, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x74, 0x6f, 0x70, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x85, 0x01, 0x82, 0xd3, 0xe4, + 0x93, 0x02, 0x7f, 0x3a, 0x01, 0x2a, 0x5a, 0x42, 0x3a, 0x01, 0x2a, 0x22, 0x3d, 0x2f, 0x61, 0x70, + 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, + 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x62, 0x61, 0x74, 0x63, + 0x68, 0x2d, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x7b, 0x6a, 0x6f, + 0x62, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x73, 0x74, 0x6f, 0x70, 0x22, 0x36, 0x2f, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x7d, 0x2f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x2d, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x7b, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x7d, 0x2f, 0x73, 0x74, + 0x6f, 0x70, 0x12, 0x90, 0x02, 0x0a, 0x16, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x42, + 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3e, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3f, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x75, + 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x6f, 0x5a, 0x3a, 0x12, 0x38, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, + 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x2d, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x7b, 0x6a, 0x6f, 0x62, 0x5f, 0x69, + 0x64, 0x7d, 0x12, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, + 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x62, 0x61, 0x74, 0x63, + 0x68, 0x2d, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x7b, 0x6a, 0x6f, + 0x62, 0x5f, 0x69, 0x64, 0x7d, 0x12, 0xf5, 0x01, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x61, + 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x3b, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3c, 0x2e, 0x74, 0x65, 0x6d, + 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, + 0x74, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x63, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x5d, + 0x5a, 0x31, 0x12, 0x2f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x7d, 0x2f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x2d, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x12, 0x28, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, + 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x2d, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x8f, 0x01, + 0x0a, 0x12, 0x50, 0x6f, 0x6c, 0x6c, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x54, 0x61, 0x73, 0x6b, 0x51, + 0x75, 0x65, 0x75, 0x65, 0x12, 0x3a, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6f, 0x6c, 0x6c, 0x4e, 0x65, 0x78, 0x75, 0x73, + 0x54, 0x61, 0x73, 0x6b, 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x3b, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x6f, 0x6c, 0x6c, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x54, 0x61, 0x73, 0x6b, + 0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0xa4, 0x01, 0x0a, 0x19, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x4e, 0x65, 0x78, 0x75, 0x73, + 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x12, 0x41, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x54, 0x61, 0x73, 0x6b, + 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x42, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x54, + 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x9b, 0x01, 0x0a, 0x16, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x64, 0x4e, 0x65, 0x78, 0x75, 0x73, 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, + 0x64, 0x12, 0x3e, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x4e, 0x65, 0x78, 0x75, 0x73, + 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x3f, 0x2e, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x64, 0x4e, 0x65, 0x78, 0x75, 0x73, + 0x54, 0x61, 0x73, 0x6b, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x42, 0xb6, 0x01, 0x0a, 0x22, 0x69, 0x6f, 0x2e, 0x74, 0x65, 0x6d, 0x70, + 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x53, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x35, 0x67, 0x6f, 0x2e, + 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x2e, 0x69, 0x6f, 0x2f, 0x61, 0x70, 0x69, 0x2f, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, + 0x76, 0x31, 0x3b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0xaa, 0x02, 0x21, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x6c, 0x69, 0x6f, 0x2e, + 0x41, 0x70, 0x69, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x2e, 0x56, 0x31, 0xea, 0x02, 0x24, 0x54, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x6c, 0x69, 0x6f, 0x3a, 0x3a, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var file_temporal_api_workflowservice_v1_service_proto_goTypes = []any{ + (*RegisterNamespaceRequest)(nil), // 0: temporal.api.workflowservice.v1.RegisterNamespaceRequest + (*DescribeNamespaceRequest)(nil), // 1: temporal.api.workflowservice.v1.DescribeNamespaceRequest + (*ListNamespacesRequest)(nil), // 2: temporal.api.workflowservice.v1.ListNamespacesRequest + (*UpdateNamespaceRequest)(nil), // 3: temporal.api.workflowservice.v1.UpdateNamespaceRequest + (*DeprecateNamespaceRequest)(nil), // 4: temporal.api.workflowservice.v1.DeprecateNamespaceRequest + (*StartWorkflowExecutionRequest)(nil), // 5: temporal.api.workflowservice.v1.StartWorkflowExecutionRequest + (*ExecuteMultiOperationRequest)(nil), // 6: temporal.api.workflowservice.v1.ExecuteMultiOperationRequest + (*GetWorkflowExecutionHistoryRequest)(nil), // 7: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryRequest + (*GetWorkflowExecutionHistoryReverseRequest)(nil), // 8: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryReverseRequest + (*PollWorkflowTaskQueueRequest)(nil), // 9: temporal.api.workflowservice.v1.PollWorkflowTaskQueueRequest + (*RespondWorkflowTaskCompletedRequest)(nil), // 10: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest + (*RespondWorkflowTaskFailedRequest)(nil), // 11: temporal.api.workflowservice.v1.RespondWorkflowTaskFailedRequest + (*PollActivityTaskQueueRequest)(nil), // 12: temporal.api.workflowservice.v1.PollActivityTaskQueueRequest + (*RecordActivityTaskHeartbeatRequest)(nil), // 13: temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatRequest + (*RecordActivityTaskHeartbeatByIdRequest)(nil), // 14: temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatByIdRequest + (*RespondActivityTaskCompletedRequest)(nil), // 15: temporal.api.workflowservice.v1.RespondActivityTaskCompletedRequest + (*RespondActivityTaskCompletedByIdRequest)(nil), // 16: temporal.api.workflowservice.v1.RespondActivityTaskCompletedByIdRequest + (*RespondActivityTaskFailedRequest)(nil), // 17: temporal.api.workflowservice.v1.RespondActivityTaskFailedRequest + (*RespondActivityTaskFailedByIdRequest)(nil), // 18: temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdRequest + (*RespondActivityTaskCanceledRequest)(nil), // 19: temporal.api.workflowservice.v1.RespondActivityTaskCanceledRequest + (*RespondActivityTaskCanceledByIdRequest)(nil), // 20: temporal.api.workflowservice.v1.RespondActivityTaskCanceledByIdRequest + (*RequestCancelWorkflowExecutionRequest)(nil), // 21: temporal.api.workflowservice.v1.RequestCancelWorkflowExecutionRequest + (*SignalWorkflowExecutionRequest)(nil), // 22: temporal.api.workflowservice.v1.SignalWorkflowExecutionRequest + (*SignalWithStartWorkflowExecutionRequest)(nil), // 23: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest + (*ResetWorkflowExecutionRequest)(nil), // 24: temporal.api.workflowservice.v1.ResetWorkflowExecutionRequest + (*TerminateWorkflowExecutionRequest)(nil), // 25: temporal.api.workflowservice.v1.TerminateWorkflowExecutionRequest + (*DeleteWorkflowExecutionRequest)(nil), // 26: temporal.api.workflowservice.v1.DeleteWorkflowExecutionRequest + (*ListOpenWorkflowExecutionsRequest)(nil), // 27: temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsRequest + (*ListClosedWorkflowExecutionsRequest)(nil), // 28: temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsRequest + (*ListWorkflowExecutionsRequest)(nil), // 29: temporal.api.workflowservice.v1.ListWorkflowExecutionsRequest + (*ListArchivedWorkflowExecutionsRequest)(nil), // 30: temporal.api.workflowservice.v1.ListArchivedWorkflowExecutionsRequest + (*ScanWorkflowExecutionsRequest)(nil), // 31: temporal.api.workflowservice.v1.ScanWorkflowExecutionsRequest + (*CountWorkflowExecutionsRequest)(nil), // 32: temporal.api.workflowservice.v1.CountWorkflowExecutionsRequest + (*GetSearchAttributesRequest)(nil), // 33: temporal.api.workflowservice.v1.GetSearchAttributesRequest + (*RespondQueryTaskCompletedRequest)(nil), // 34: temporal.api.workflowservice.v1.RespondQueryTaskCompletedRequest + (*ResetStickyTaskQueueRequest)(nil), // 35: temporal.api.workflowservice.v1.ResetStickyTaskQueueRequest + (*QueryWorkflowRequest)(nil), // 36: temporal.api.workflowservice.v1.QueryWorkflowRequest + (*DescribeWorkflowExecutionRequest)(nil), // 37: temporal.api.workflowservice.v1.DescribeWorkflowExecutionRequest + (*DescribeTaskQueueRequest)(nil), // 38: temporal.api.workflowservice.v1.DescribeTaskQueueRequest + (*GetClusterInfoRequest)(nil), // 39: temporal.api.workflowservice.v1.GetClusterInfoRequest + (*GetSystemInfoRequest)(nil), // 40: temporal.api.workflowservice.v1.GetSystemInfoRequest + (*ListTaskQueuePartitionsRequest)(nil), // 41: temporal.api.workflowservice.v1.ListTaskQueuePartitionsRequest + (*CreateScheduleRequest)(nil), // 42: temporal.api.workflowservice.v1.CreateScheduleRequest + (*DescribeScheduleRequest)(nil), // 43: temporal.api.workflowservice.v1.DescribeScheduleRequest + (*UpdateScheduleRequest)(nil), // 44: temporal.api.workflowservice.v1.UpdateScheduleRequest + (*PatchScheduleRequest)(nil), // 45: temporal.api.workflowservice.v1.PatchScheduleRequest + (*ListScheduleMatchingTimesRequest)(nil), // 46: temporal.api.workflowservice.v1.ListScheduleMatchingTimesRequest + (*DeleteScheduleRequest)(nil), // 47: temporal.api.workflowservice.v1.DeleteScheduleRequest + (*ListSchedulesRequest)(nil), // 48: temporal.api.workflowservice.v1.ListSchedulesRequest + (*UpdateWorkerBuildIdCompatibilityRequest)(nil), // 49: temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest + (*GetWorkerBuildIdCompatibilityRequest)(nil), // 50: temporal.api.workflowservice.v1.GetWorkerBuildIdCompatibilityRequest + (*UpdateWorkerVersioningRulesRequest)(nil), // 51: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest + (*GetWorkerVersioningRulesRequest)(nil), // 52: temporal.api.workflowservice.v1.GetWorkerVersioningRulesRequest + (*GetWorkerTaskReachabilityRequest)(nil), // 53: temporal.api.workflowservice.v1.GetWorkerTaskReachabilityRequest + (*UpdateWorkflowExecutionRequest)(nil), // 54: temporal.api.workflowservice.v1.UpdateWorkflowExecutionRequest + (*PollWorkflowExecutionUpdateRequest)(nil), // 55: temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateRequest + (*StartBatchOperationRequest)(nil), // 56: temporal.api.workflowservice.v1.StartBatchOperationRequest + (*StopBatchOperationRequest)(nil), // 57: temporal.api.workflowservice.v1.StopBatchOperationRequest + (*DescribeBatchOperationRequest)(nil), // 58: temporal.api.workflowservice.v1.DescribeBatchOperationRequest + (*ListBatchOperationsRequest)(nil), // 59: temporal.api.workflowservice.v1.ListBatchOperationsRequest + (*PollNexusTaskQueueRequest)(nil), // 60: temporal.api.workflowservice.v1.PollNexusTaskQueueRequest + (*RespondNexusTaskCompletedRequest)(nil), // 61: temporal.api.workflowservice.v1.RespondNexusTaskCompletedRequest + (*RespondNexusTaskFailedRequest)(nil), // 62: temporal.api.workflowservice.v1.RespondNexusTaskFailedRequest + (*RegisterNamespaceResponse)(nil), // 63: temporal.api.workflowservice.v1.RegisterNamespaceResponse + (*DescribeNamespaceResponse)(nil), // 64: temporal.api.workflowservice.v1.DescribeNamespaceResponse + (*ListNamespacesResponse)(nil), // 65: temporal.api.workflowservice.v1.ListNamespacesResponse + (*UpdateNamespaceResponse)(nil), // 66: temporal.api.workflowservice.v1.UpdateNamespaceResponse + (*DeprecateNamespaceResponse)(nil), // 67: temporal.api.workflowservice.v1.DeprecateNamespaceResponse + (*StartWorkflowExecutionResponse)(nil), // 68: temporal.api.workflowservice.v1.StartWorkflowExecutionResponse + (*ExecuteMultiOperationResponse)(nil), // 69: temporal.api.workflowservice.v1.ExecuteMultiOperationResponse + (*GetWorkflowExecutionHistoryResponse)(nil), // 70: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryResponse + (*GetWorkflowExecutionHistoryReverseResponse)(nil), // 71: temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryReverseResponse + (*PollWorkflowTaskQueueResponse)(nil), // 72: temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse + (*RespondWorkflowTaskCompletedResponse)(nil), // 73: temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedResponse + (*RespondWorkflowTaskFailedResponse)(nil), // 74: temporal.api.workflowservice.v1.RespondWorkflowTaskFailedResponse + (*PollActivityTaskQueueResponse)(nil), // 75: temporal.api.workflowservice.v1.PollActivityTaskQueueResponse + (*RecordActivityTaskHeartbeatResponse)(nil), // 76: temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatResponse + (*RecordActivityTaskHeartbeatByIdResponse)(nil), // 77: temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatByIdResponse + (*RespondActivityTaskCompletedResponse)(nil), // 78: temporal.api.workflowservice.v1.RespondActivityTaskCompletedResponse + (*RespondActivityTaskCompletedByIdResponse)(nil), // 79: temporal.api.workflowservice.v1.RespondActivityTaskCompletedByIdResponse + (*RespondActivityTaskFailedResponse)(nil), // 80: temporal.api.workflowservice.v1.RespondActivityTaskFailedResponse + (*RespondActivityTaskFailedByIdResponse)(nil), // 81: temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdResponse + (*RespondActivityTaskCanceledResponse)(nil), // 82: temporal.api.workflowservice.v1.RespondActivityTaskCanceledResponse + (*RespondActivityTaskCanceledByIdResponse)(nil), // 83: temporal.api.workflowservice.v1.RespondActivityTaskCanceledByIdResponse + (*RequestCancelWorkflowExecutionResponse)(nil), // 84: temporal.api.workflowservice.v1.RequestCancelWorkflowExecutionResponse + (*SignalWorkflowExecutionResponse)(nil), // 85: temporal.api.workflowservice.v1.SignalWorkflowExecutionResponse + (*SignalWithStartWorkflowExecutionResponse)(nil), // 86: temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionResponse + (*ResetWorkflowExecutionResponse)(nil), // 87: temporal.api.workflowservice.v1.ResetWorkflowExecutionResponse + (*TerminateWorkflowExecutionResponse)(nil), // 88: temporal.api.workflowservice.v1.TerminateWorkflowExecutionResponse + (*DeleteWorkflowExecutionResponse)(nil), // 89: temporal.api.workflowservice.v1.DeleteWorkflowExecutionResponse + (*ListOpenWorkflowExecutionsResponse)(nil), // 90: temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsResponse + (*ListClosedWorkflowExecutionsResponse)(nil), // 91: temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsResponse + (*ListWorkflowExecutionsResponse)(nil), // 92: temporal.api.workflowservice.v1.ListWorkflowExecutionsResponse + (*ListArchivedWorkflowExecutionsResponse)(nil), // 93: temporal.api.workflowservice.v1.ListArchivedWorkflowExecutionsResponse + (*ScanWorkflowExecutionsResponse)(nil), // 94: temporal.api.workflowservice.v1.ScanWorkflowExecutionsResponse + (*CountWorkflowExecutionsResponse)(nil), // 95: temporal.api.workflowservice.v1.CountWorkflowExecutionsResponse + (*GetSearchAttributesResponse)(nil), // 96: temporal.api.workflowservice.v1.GetSearchAttributesResponse + (*RespondQueryTaskCompletedResponse)(nil), // 97: temporal.api.workflowservice.v1.RespondQueryTaskCompletedResponse + (*ResetStickyTaskQueueResponse)(nil), // 98: temporal.api.workflowservice.v1.ResetStickyTaskQueueResponse + (*QueryWorkflowResponse)(nil), // 99: temporal.api.workflowservice.v1.QueryWorkflowResponse + (*DescribeWorkflowExecutionResponse)(nil), // 100: temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse + (*DescribeTaskQueueResponse)(nil), // 101: temporal.api.workflowservice.v1.DescribeTaskQueueResponse + (*GetClusterInfoResponse)(nil), // 102: temporal.api.workflowservice.v1.GetClusterInfoResponse + (*GetSystemInfoResponse)(nil), // 103: temporal.api.workflowservice.v1.GetSystemInfoResponse + (*ListTaskQueuePartitionsResponse)(nil), // 104: temporal.api.workflowservice.v1.ListTaskQueuePartitionsResponse + (*CreateScheduleResponse)(nil), // 105: temporal.api.workflowservice.v1.CreateScheduleResponse + (*DescribeScheduleResponse)(nil), // 106: temporal.api.workflowservice.v1.DescribeScheduleResponse + (*UpdateScheduleResponse)(nil), // 107: temporal.api.workflowservice.v1.UpdateScheduleResponse + (*PatchScheduleResponse)(nil), // 108: temporal.api.workflowservice.v1.PatchScheduleResponse + (*ListScheduleMatchingTimesResponse)(nil), // 109: temporal.api.workflowservice.v1.ListScheduleMatchingTimesResponse + (*DeleteScheduleResponse)(nil), // 110: temporal.api.workflowservice.v1.DeleteScheduleResponse + (*ListSchedulesResponse)(nil), // 111: temporal.api.workflowservice.v1.ListSchedulesResponse + (*UpdateWorkerBuildIdCompatibilityResponse)(nil), // 112: temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityResponse + (*GetWorkerBuildIdCompatibilityResponse)(nil), // 113: temporal.api.workflowservice.v1.GetWorkerBuildIdCompatibilityResponse + (*UpdateWorkerVersioningRulesResponse)(nil), // 114: temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesResponse + (*GetWorkerVersioningRulesResponse)(nil), // 115: temporal.api.workflowservice.v1.GetWorkerVersioningRulesResponse + (*GetWorkerTaskReachabilityResponse)(nil), // 116: temporal.api.workflowservice.v1.GetWorkerTaskReachabilityResponse + (*UpdateWorkflowExecutionResponse)(nil), // 117: temporal.api.workflowservice.v1.UpdateWorkflowExecutionResponse + (*PollWorkflowExecutionUpdateResponse)(nil), // 118: temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateResponse + (*StartBatchOperationResponse)(nil), // 119: temporal.api.workflowservice.v1.StartBatchOperationResponse + (*StopBatchOperationResponse)(nil), // 120: temporal.api.workflowservice.v1.StopBatchOperationResponse + (*DescribeBatchOperationResponse)(nil), // 121: temporal.api.workflowservice.v1.DescribeBatchOperationResponse + (*ListBatchOperationsResponse)(nil), // 122: temporal.api.workflowservice.v1.ListBatchOperationsResponse + (*PollNexusTaskQueueResponse)(nil), // 123: temporal.api.workflowservice.v1.PollNexusTaskQueueResponse + (*RespondNexusTaskCompletedResponse)(nil), // 124: temporal.api.workflowservice.v1.RespondNexusTaskCompletedResponse + (*RespondNexusTaskFailedResponse)(nil), // 125: temporal.api.workflowservice.v1.RespondNexusTaskFailedResponse +} +var file_temporal_api_workflowservice_v1_service_proto_depIdxs = []int32{ + 0, // 0: temporal.api.workflowservice.v1.WorkflowService.RegisterNamespace:input_type -> temporal.api.workflowservice.v1.RegisterNamespaceRequest + 1, // 1: temporal.api.workflowservice.v1.WorkflowService.DescribeNamespace:input_type -> temporal.api.workflowservice.v1.DescribeNamespaceRequest + 2, // 2: temporal.api.workflowservice.v1.WorkflowService.ListNamespaces:input_type -> temporal.api.workflowservice.v1.ListNamespacesRequest + 3, // 3: temporal.api.workflowservice.v1.WorkflowService.UpdateNamespace:input_type -> temporal.api.workflowservice.v1.UpdateNamespaceRequest + 4, // 4: temporal.api.workflowservice.v1.WorkflowService.DeprecateNamespace:input_type -> temporal.api.workflowservice.v1.DeprecateNamespaceRequest + 5, // 5: temporal.api.workflowservice.v1.WorkflowService.StartWorkflowExecution:input_type -> temporal.api.workflowservice.v1.StartWorkflowExecutionRequest + 6, // 6: temporal.api.workflowservice.v1.WorkflowService.ExecuteMultiOperation:input_type -> temporal.api.workflowservice.v1.ExecuteMultiOperationRequest + 7, // 7: temporal.api.workflowservice.v1.WorkflowService.GetWorkflowExecutionHistory:input_type -> temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryRequest + 8, // 8: temporal.api.workflowservice.v1.WorkflowService.GetWorkflowExecutionHistoryReverse:input_type -> temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryReverseRequest + 9, // 9: temporal.api.workflowservice.v1.WorkflowService.PollWorkflowTaskQueue:input_type -> temporal.api.workflowservice.v1.PollWorkflowTaskQueueRequest + 10, // 10: temporal.api.workflowservice.v1.WorkflowService.RespondWorkflowTaskCompleted:input_type -> temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest + 11, // 11: temporal.api.workflowservice.v1.WorkflowService.RespondWorkflowTaskFailed:input_type -> temporal.api.workflowservice.v1.RespondWorkflowTaskFailedRequest + 12, // 12: temporal.api.workflowservice.v1.WorkflowService.PollActivityTaskQueue:input_type -> temporal.api.workflowservice.v1.PollActivityTaskQueueRequest + 13, // 13: temporal.api.workflowservice.v1.WorkflowService.RecordActivityTaskHeartbeat:input_type -> temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatRequest + 14, // 14: temporal.api.workflowservice.v1.WorkflowService.RecordActivityTaskHeartbeatById:input_type -> temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatByIdRequest + 15, // 15: temporal.api.workflowservice.v1.WorkflowService.RespondActivityTaskCompleted:input_type -> temporal.api.workflowservice.v1.RespondActivityTaskCompletedRequest + 16, // 16: temporal.api.workflowservice.v1.WorkflowService.RespondActivityTaskCompletedById:input_type -> temporal.api.workflowservice.v1.RespondActivityTaskCompletedByIdRequest + 17, // 17: temporal.api.workflowservice.v1.WorkflowService.RespondActivityTaskFailed:input_type -> temporal.api.workflowservice.v1.RespondActivityTaskFailedRequest + 18, // 18: temporal.api.workflowservice.v1.WorkflowService.RespondActivityTaskFailedById:input_type -> temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdRequest + 19, // 19: temporal.api.workflowservice.v1.WorkflowService.RespondActivityTaskCanceled:input_type -> temporal.api.workflowservice.v1.RespondActivityTaskCanceledRequest + 20, // 20: temporal.api.workflowservice.v1.WorkflowService.RespondActivityTaskCanceledById:input_type -> temporal.api.workflowservice.v1.RespondActivityTaskCanceledByIdRequest + 21, // 21: temporal.api.workflowservice.v1.WorkflowService.RequestCancelWorkflowExecution:input_type -> temporal.api.workflowservice.v1.RequestCancelWorkflowExecutionRequest + 22, // 22: temporal.api.workflowservice.v1.WorkflowService.SignalWorkflowExecution:input_type -> temporal.api.workflowservice.v1.SignalWorkflowExecutionRequest + 23, // 23: temporal.api.workflowservice.v1.WorkflowService.SignalWithStartWorkflowExecution:input_type -> temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionRequest + 24, // 24: temporal.api.workflowservice.v1.WorkflowService.ResetWorkflowExecution:input_type -> temporal.api.workflowservice.v1.ResetWorkflowExecutionRequest + 25, // 25: temporal.api.workflowservice.v1.WorkflowService.TerminateWorkflowExecution:input_type -> temporal.api.workflowservice.v1.TerminateWorkflowExecutionRequest + 26, // 26: temporal.api.workflowservice.v1.WorkflowService.DeleteWorkflowExecution:input_type -> temporal.api.workflowservice.v1.DeleteWorkflowExecutionRequest + 27, // 27: temporal.api.workflowservice.v1.WorkflowService.ListOpenWorkflowExecutions:input_type -> temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsRequest + 28, // 28: temporal.api.workflowservice.v1.WorkflowService.ListClosedWorkflowExecutions:input_type -> temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsRequest + 29, // 29: temporal.api.workflowservice.v1.WorkflowService.ListWorkflowExecutions:input_type -> temporal.api.workflowservice.v1.ListWorkflowExecutionsRequest + 30, // 30: temporal.api.workflowservice.v1.WorkflowService.ListArchivedWorkflowExecutions:input_type -> temporal.api.workflowservice.v1.ListArchivedWorkflowExecutionsRequest + 31, // 31: temporal.api.workflowservice.v1.WorkflowService.ScanWorkflowExecutions:input_type -> temporal.api.workflowservice.v1.ScanWorkflowExecutionsRequest + 32, // 32: temporal.api.workflowservice.v1.WorkflowService.CountWorkflowExecutions:input_type -> temporal.api.workflowservice.v1.CountWorkflowExecutionsRequest + 33, // 33: temporal.api.workflowservice.v1.WorkflowService.GetSearchAttributes:input_type -> temporal.api.workflowservice.v1.GetSearchAttributesRequest + 34, // 34: temporal.api.workflowservice.v1.WorkflowService.RespondQueryTaskCompleted:input_type -> temporal.api.workflowservice.v1.RespondQueryTaskCompletedRequest + 35, // 35: temporal.api.workflowservice.v1.WorkflowService.ResetStickyTaskQueue:input_type -> temporal.api.workflowservice.v1.ResetStickyTaskQueueRequest + 36, // 36: temporal.api.workflowservice.v1.WorkflowService.QueryWorkflow:input_type -> temporal.api.workflowservice.v1.QueryWorkflowRequest + 37, // 37: temporal.api.workflowservice.v1.WorkflowService.DescribeWorkflowExecution:input_type -> temporal.api.workflowservice.v1.DescribeWorkflowExecutionRequest + 38, // 38: temporal.api.workflowservice.v1.WorkflowService.DescribeTaskQueue:input_type -> temporal.api.workflowservice.v1.DescribeTaskQueueRequest + 39, // 39: temporal.api.workflowservice.v1.WorkflowService.GetClusterInfo:input_type -> temporal.api.workflowservice.v1.GetClusterInfoRequest + 40, // 40: temporal.api.workflowservice.v1.WorkflowService.GetSystemInfo:input_type -> temporal.api.workflowservice.v1.GetSystemInfoRequest + 41, // 41: temporal.api.workflowservice.v1.WorkflowService.ListTaskQueuePartitions:input_type -> temporal.api.workflowservice.v1.ListTaskQueuePartitionsRequest + 42, // 42: temporal.api.workflowservice.v1.WorkflowService.CreateSchedule:input_type -> temporal.api.workflowservice.v1.CreateScheduleRequest + 43, // 43: temporal.api.workflowservice.v1.WorkflowService.DescribeSchedule:input_type -> temporal.api.workflowservice.v1.DescribeScheduleRequest + 44, // 44: temporal.api.workflowservice.v1.WorkflowService.UpdateSchedule:input_type -> temporal.api.workflowservice.v1.UpdateScheduleRequest + 45, // 45: temporal.api.workflowservice.v1.WorkflowService.PatchSchedule:input_type -> temporal.api.workflowservice.v1.PatchScheduleRequest + 46, // 46: temporal.api.workflowservice.v1.WorkflowService.ListScheduleMatchingTimes:input_type -> temporal.api.workflowservice.v1.ListScheduleMatchingTimesRequest + 47, // 47: temporal.api.workflowservice.v1.WorkflowService.DeleteSchedule:input_type -> temporal.api.workflowservice.v1.DeleteScheduleRequest + 48, // 48: temporal.api.workflowservice.v1.WorkflowService.ListSchedules:input_type -> temporal.api.workflowservice.v1.ListSchedulesRequest + 49, // 49: temporal.api.workflowservice.v1.WorkflowService.UpdateWorkerBuildIdCompatibility:input_type -> temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityRequest + 50, // 50: temporal.api.workflowservice.v1.WorkflowService.GetWorkerBuildIdCompatibility:input_type -> temporal.api.workflowservice.v1.GetWorkerBuildIdCompatibilityRequest + 51, // 51: temporal.api.workflowservice.v1.WorkflowService.UpdateWorkerVersioningRules:input_type -> temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesRequest + 52, // 52: temporal.api.workflowservice.v1.WorkflowService.GetWorkerVersioningRules:input_type -> temporal.api.workflowservice.v1.GetWorkerVersioningRulesRequest + 53, // 53: temporal.api.workflowservice.v1.WorkflowService.GetWorkerTaskReachability:input_type -> temporal.api.workflowservice.v1.GetWorkerTaskReachabilityRequest + 54, // 54: temporal.api.workflowservice.v1.WorkflowService.UpdateWorkflowExecution:input_type -> temporal.api.workflowservice.v1.UpdateWorkflowExecutionRequest + 55, // 55: temporal.api.workflowservice.v1.WorkflowService.PollWorkflowExecutionUpdate:input_type -> temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateRequest + 56, // 56: temporal.api.workflowservice.v1.WorkflowService.StartBatchOperation:input_type -> temporal.api.workflowservice.v1.StartBatchOperationRequest + 57, // 57: temporal.api.workflowservice.v1.WorkflowService.StopBatchOperation:input_type -> temporal.api.workflowservice.v1.StopBatchOperationRequest + 58, // 58: temporal.api.workflowservice.v1.WorkflowService.DescribeBatchOperation:input_type -> temporal.api.workflowservice.v1.DescribeBatchOperationRequest + 59, // 59: temporal.api.workflowservice.v1.WorkflowService.ListBatchOperations:input_type -> temporal.api.workflowservice.v1.ListBatchOperationsRequest + 60, // 60: temporal.api.workflowservice.v1.WorkflowService.PollNexusTaskQueue:input_type -> temporal.api.workflowservice.v1.PollNexusTaskQueueRequest + 61, // 61: temporal.api.workflowservice.v1.WorkflowService.RespondNexusTaskCompleted:input_type -> temporal.api.workflowservice.v1.RespondNexusTaskCompletedRequest + 62, // 62: temporal.api.workflowservice.v1.WorkflowService.RespondNexusTaskFailed:input_type -> temporal.api.workflowservice.v1.RespondNexusTaskFailedRequest + 63, // 63: temporal.api.workflowservice.v1.WorkflowService.RegisterNamespace:output_type -> temporal.api.workflowservice.v1.RegisterNamespaceResponse + 64, // 64: temporal.api.workflowservice.v1.WorkflowService.DescribeNamespace:output_type -> temporal.api.workflowservice.v1.DescribeNamespaceResponse + 65, // 65: temporal.api.workflowservice.v1.WorkflowService.ListNamespaces:output_type -> temporal.api.workflowservice.v1.ListNamespacesResponse + 66, // 66: temporal.api.workflowservice.v1.WorkflowService.UpdateNamespace:output_type -> temporal.api.workflowservice.v1.UpdateNamespaceResponse + 67, // 67: temporal.api.workflowservice.v1.WorkflowService.DeprecateNamespace:output_type -> temporal.api.workflowservice.v1.DeprecateNamespaceResponse + 68, // 68: temporal.api.workflowservice.v1.WorkflowService.StartWorkflowExecution:output_type -> temporal.api.workflowservice.v1.StartWorkflowExecutionResponse + 69, // 69: temporal.api.workflowservice.v1.WorkflowService.ExecuteMultiOperation:output_type -> temporal.api.workflowservice.v1.ExecuteMultiOperationResponse + 70, // 70: temporal.api.workflowservice.v1.WorkflowService.GetWorkflowExecutionHistory:output_type -> temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryResponse + 71, // 71: temporal.api.workflowservice.v1.WorkflowService.GetWorkflowExecutionHistoryReverse:output_type -> temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryReverseResponse + 72, // 72: temporal.api.workflowservice.v1.WorkflowService.PollWorkflowTaskQueue:output_type -> temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse + 73, // 73: temporal.api.workflowservice.v1.WorkflowService.RespondWorkflowTaskCompleted:output_type -> temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedResponse + 74, // 74: temporal.api.workflowservice.v1.WorkflowService.RespondWorkflowTaskFailed:output_type -> temporal.api.workflowservice.v1.RespondWorkflowTaskFailedResponse + 75, // 75: temporal.api.workflowservice.v1.WorkflowService.PollActivityTaskQueue:output_type -> temporal.api.workflowservice.v1.PollActivityTaskQueueResponse + 76, // 76: temporal.api.workflowservice.v1.WorkflowService.RecordActivityTaskHeartbeat:output_type -> temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatResponse + 77, // 77: temporal.api.workflowservice.v1.WorkflowService.RecordActivityTaskHeartbeatById:output_type -> temporal.api.workflowservice.v1.RecordActivityTaskHeartbeatByIdResponse + 78, // 78: temporal.api.workflowservice.v1.WorkflowService.RespondActivityTaskCompleted:output_type -> temporal.api.workflowservice.v1.RespondActivityTaskCompletedResponse + 79, // 79: temporal.api.workflowservice.v1.WorkflowService.RespondActivityTaskCompletedById:output_type -> temporal.api.workflowservice.v1.RespondActivityTaskCompletedByIdResponse + 80, // 80: temporal.api.workflowservice.v1.WorkflowService.RespondActivityTaskFailed:output_type -> temporal.api.workflowservice.v1.RespondActivityTaskFailedResponse + 81, // 81: temporal.api.workflowservice.v1.WorkflowService.RespondActivityTaskFailedById:output_type -> temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdResponse + 82, // 82: temporal.api.workflowservice.v1.WorkflowService.RespondActivityTaskCanceled:output_type -> temporal.api.workflowservice.v1.RespondActivityTaskCanceledResponse + 83, // 83: temporal.api.workflowservice.v1.WorkflowService.RespondActivityTaskCanceledById:output_type -> temporal.api.workflowservice.v1.RespondActivityTaskCanceledByIdResponse + 84, // 84: temporal.api.workflowservice.v1.WorkflowService.RequestCancelWorkflowExecution:output_type -> temporal.api.workflowservice.v1.RequestCancelWorkflowExecutionResponse + 85, // 85: temporal.api.workflowservice.v1.WorkflowService.SignalWorkflowExecution:output_type -> temporal.api.workflowservice.v1.SignalWorkflowExecutionResponse + 86, // 86: temporal.api.workflowservice.v1.WorkflowService.SignalWithStartWorkflowExecution:output_type -> temporal.api.workflowservice.v1.SignalWithStartWorkflowExecutionResponse + 87, // 87: temporal.api.workflowservice.v1.WorkflowService.ResetWorkflowExecution:output_type -> temporal.api.workflowservice.v1.ResetWorkflowExecutionResponse + 88, // 88: temporal.api.workflowservice.v1.WorkflowService.TerminateWorkflowExecution:output_type -> temporal.api.workflowservice.v1.TerminateWorkflowExecutionResponse + 89, // 89: temporal.api.workflowservice.v1.WorkflowService.DeleteWorkflowExecution:output_type -> temporal.api.workflowservice.v1.DeleteWorkflowExecutionResponse + 90, // 90: temporal.api.workflowservice.v1.WorkflowService.ListOpenWorkflowExecutions:output_type -> temporal.api.workflowservice.v1.ListOpenWorkflowExecutionsResponse + 91, // 91: temporal.api.workflowservice.v1.WorkflowService.ListClosedWorkflowExecutions:output_type -> temporal.api.workflowservice.v1.ListClosedWorkflowExecutionsResponse + 92, // 92: temporal.api.workflowservice.v1.WorkflowService.ListWorkflowExecutions:output_type -> temporal.api.workflowservice.v1.ListWorkflowExecutionsResponse + 93, // 93: temporal.api.workflowservice.v1.WorkflowService.ListArchivedWorkflowExecutions:output_type -> temporal.api.workflowservice.v1.ListArchivedWorkflowExecutionsResponse + 94, // 94: temporal.api.workflowservice.v1.WorkflowService.ScanWorkflowExecutions:output_type -> temporal.api.workflowservice.v1.ScanWorkflowExecutionsResponse + 95, // 95: temporal.api.workflowservice.v1.WorkflowService.CountWorkflowExecutions:output_type -> temporal.api.workflowservice.v1.CountWorkflowExecutionsResponse + 96, // 96: temporal.api.workflowservice.v1.WorkflowService.GetSearchAttributes:output_type -> temporal.api.workflowservice.v1.GetSearchAttributesResponse + 97, // 97: temporal.api.workflowservice.v1.WorkflowService.RespondQueryTaskCompleted:output_type -> temporal.api.workflowservice.v1.RespondQueryTaskCompletedResponse + 98, // 98: temporal.api.workflowservice.v1.WorkflowService.ResetStickyTaskQueue:output_type -> temporal.api.workflowservice.v1.ResetStickyTaskQueueResponse + 99, // 99: temporal.api.workflowservice.v1.WorkflowService.QueryWorkflow:output_type -> temporal.api.workflowservice.v1.QueryWorkflowResponse + 100, // 100: temporal.api.workflowservice.v1.WorkflowService.DescribeWorkflowExecution:output_type -> temporal.api.workflowservice.v1.DescribeWorkflowExecutionResponse + 101, // 101: temporal.api.workflowservice.v1.WorkflowService.DescribeTaskQueue:output_type -> temporal.api.workflowservice.v1.DescribeTaskQueueResponse + 102, // 102: temporal.api.workflowservice.v1.WorkflowService.GetClusterInfo:output_type -> temporal.api.workflowservice.v1.GetClusterInfoResponse + 103, // 103: temporal.api.workflowservice.v1.WorkflowService.GetSystemInfo:output_type -> temporal.api.workflowservice.v1.GetSystemInfoResponse + 104, // 104: temporal.api.workflowservice.v1.WorkflowService.ListTaskQueuePartitions:output_type -> temporal.api.workflowservice.v1.ListTaskQueuePartitionsResponse + 105, // 105: temporal.api.workflowservice.v1.WorkflowService.CreateSchedule:output_type -> temporal.api.workflowservice.v1.CreateScheduleResponse + 106, // 106: temporal.api.workflowservice.v1.WorkflowService.DescribeSchedule:output_type -> temporal.api.workflowservice.v1.DescribeScheduleResponse + 107, // 107: temporal.api.workflowservice.v1.WorkflowService.UpdateSchedule:output_type -> temporal.api.workflowservice.v1.UpdateScheduleResponse + 108, // 108: temporal.api.workflowservice.v1.WorkflowService.PatchSchedule:output_type -> temporal.api.workflowservice.v1.PatchScheduleResponse + 109, // 109: temporal.api.workflowservice.v1.WorkflowService.ListScheduleMatchingTimes:output_type -> temporal.api.workflowservice.v1.ListScheduleMatchingTimesResponse + 110, // 110: temporal.api.workflowservice.v1.WorkflowService.DeleteSchedule:output_type -> temporal.api.workflowservice.v1.DeleteScheduleResponse + 111, // 111: temporal.api.workflowservice.v1.WorkflowService.ListSchedules:output_type -> temporal.api.workflowservice.v1.ListSchedulesResponse + 112, // 112: temporal.api.workflowservice.v1.WorkflowService.UpdateWorkerBuildIdCompatibility:output_type -> temporal.api.workflowservice.v1.UpdateWorkerBuildIdCompatibilityResponse + 113, // 113: temporal.api.workflowservice.v1.WorkflowService.GetWorkerBuildIdCompatibility:output_type -> temporal.api.workflowservice.v1.GetWorkerBuildIdCompatibilityResponse + 114, // 114: temporal.api.workflowservice.v1.WorkflowService.UpdateWorkerVersioningRules:output_type -> temporal.api.workflowservice.v1.UpdateWorkerVersioningRulesResponse + 115, // 115: temporal.api.workflowservice.v1.WorkflowService.GetWorkerVersioningRules:output_type -> temporal.api.workflowservice.v1.GetWorkerVersioningRulesResponse + 116, // 116: temporal.api.workflowservice.v1.WorkflowService.GetWorkerTaskReachability:output_type -> temporal.api.workflowservice.v1.GetWorkerTaskReachabilityResponse + 117, // 117: temporal.api.workflowservice.v1.WorkflowService.UpdateWorkflowExecution:output_type -> temporal.api.workflowservice.v1.UpdateWorkflowExecutionResponse + 118, // 118: temporal.api.workflowservice.v1.WorkflowService.PollWorkflowExecutionUpdate:output_type -> temporal.api.workflowservice.v1.PollWorkflowExecutionUpdateResponse + 119, // 119: temporal.api.workflowservice.v1.WorkflowService.StartBatchOperation:output_type -> temporal.api.workflowservice.v1.StartBatchOperationResponse + 120, // 120: temporal.api.workflowservice.v1.WorkflowService.StopBatchOperation:output_type -> temporal.api.workflowservice.v1.StopBatchOperationResponse + 121, // 121: temporal.api.workflowservice.v1.WorkflowService.DescribeBatchOperation:output_type -> temporal.api.workflowservice.v1.DescribeBatchOperationResponse + 122, // 122: temporal.api.workflowservice.v1.WorkflowService.ListBatchOperations:output_type -> temporal.api.workflowservice.v1.ListBatchOperationsResponse + 123, // 123: temporal.api.workflowservice.v1.WorkflowService.PollNexusTaskQueue:output_type -> temporal.api.workflowservice.v1.PollNexusTaskQueueResponse + 124, // 124: temporal.api.workflowservice.v1.WorkflowService.RespondNexusTaskCompleted:output_type -> temporal.api.workflowservice.v1.RespondNexusTaskCompletedResponse + 125, // 125: temporal.api.workflowservice.v1.WorkflowService.RespondNexusTaskFailed:output_type -> temporal.api.workflowservice.v1.RespondNexusTaskFailedResponse + 63, // [63:126] is the sub-list for method output_type + 0, // [0:63] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_temporal_api_workflowservice_v1_service_proto_init() } +func file_temporal_api_workflowservice_v1_service_proto_init() { + if File_temporal_api_workflowservice_v1_service_proto != nil { + return + } + file_temporal_api_workflowservice_v1_request_response_proto_init() + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_temporal_api_workflowservice_v1_service_proto_rawDesc, + NumEnums: 0, + NumMessages: 0, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_temporal_api_workflowservice_v1_service_proto_goTypes, + DependencyIndexes: file_temporal_api_workflowservice_v1_service_proto_depIdxs, + }.Build() + File_temporal_api_workflowservice_v1_service_proto = out.File + file_temporal_api_workflowservice_v1_service_proto_rawDesc = nil + file_temporal_api_workflowservice_v1_service_proto_goTypes = nil + file_temporal_api_workflowservice_v1_service_proto_depIdxs = nil +} diff --git a/vendor/go.temporal.io/api/workflowservice/v1/service.pb.gw.go b/vendor/go.temporal.io/api/workflowservice/v1/service.pb.gw.go new file mode 100644 index 00000000000..ab84a284133 --- /dev/null +++ b/vendor/go.temporal.io/api/workflowservice/v1/service.pb.gw.go @@ -0,0 +1,10905 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-grpc-gateway. DO NOT EDIT. +// source: temporal/api/workflowservice/v1/service.proto + +/* +Package workflowservice is a reverse proxy. + +It translates gRPC into RESTful JSON APIs. +*/ +package workflowservice + +import ( + "context" + "io" + "net/http" + + "github.com/grpc-ecosystem/grpc-gateway/v2/runtime" + "github.com/grpc-ecosystem/grpc-gateway/v2/utilities" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/grpclog" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/proto" +) + +// Suppress "imported and not used" errors +var _ codes.Code +var _ io.Reader +var _ status.Status +var _ = runtime.String +var _ = utilities.NewDoubleArray +var _ = metadata.Join + +func request_WorkflowService_RegisterNamespace_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RegisterNamespaceRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.RegisterNamespace(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RegisterNamespace_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RegisterNamespaceRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.RegisterNamespace(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RegisterNamespace_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RegisterNamespaceRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.RegisterNamespace(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RegisterNamespace_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RegisterNamespaceRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.RegisterNamespace(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_DescribeNamespace_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_DescribeNamespace_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeNamespaceRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DescribeNamespace_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DescribeNamespace(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_DescribeNamespace_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeNamespaceRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DescribeNamespace_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DescribeNamespace(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_DescribeNamespace_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_DescribeNamespace_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeNamespaceRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DescribeNamespace_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DescribeNamespace(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_DescribeNamespace_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeNamespaceRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DescribeNamespace_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DescribeNamespace(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_ListNamespaces_0 = &utilities.DoubleArray{Encoding: map[string]int{}, Base: []int(nil), Check: []int(nil)} +) + +func request_WorkflowService_ListNamespaces_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListNamespacesRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListNamespaces_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListNamespaces(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ListNamespaces_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListNamespacesRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListNamespaces_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListNamespaces(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_ListNamespaces_1 = &utilities.DoubleArray{Encoding: map[string]int{}, Base: []int(nil), Check: []int(nil)} +) + +func request_WorkflowService_ListNamespaces_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListNamespacesRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListNamespaces_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListNamespaces(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ListNamespaces_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListNamespacesRequest + var metadata runtime.ServerMetadata + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListNamespaces_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListNamespaces(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_UpdateNamespace_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateNamespaceRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.UpdateNamespace(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_UpdateNamespace_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateNamespaceRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.UpdateNamespace(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_UpdateNamespace_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateNamespaceRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.UpdateNamespace(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_UpdateNamespace_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateNamespaceRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.UpdateNamespace(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_StartWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq StartWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_id") + } + + protoReq.WorkflowId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_id", err) + } + + msg, err := client.StartWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_StartWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq StartWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_id") + } + + protoReq.WorkflowId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_id", err) + } + + msg, err := server.StartWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_StartWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq StartWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_id") + } + + protoReq.WorkflowId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_id", err) + } + + msg, err := client.StartWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_StartWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq StartWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_id") + } + + protoReq.WorkflowId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_id", err) + } + + msg, err := server.StartWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_ExecuteMultiOperation_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ExecuteMultiOperationRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.ExecuteMultiOperation(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ExecuteMultiOperation_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ExecuteMultiOperationRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.ExecuteMultiOperation(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_ExecuteMultiOperation_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ExecuteMultiOperationRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.ExecuteMultiOperation(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ExecuteMultiOperation_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ExecuteMultiOperationRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.ExecuteMultiOperation(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_GetWorkflowExecutionHistory_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "execution": 1, "workflow_id": 2}, Base: []int{1, 1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 3, 2, 4}} +) + +func request_WorkflowService_GetWorkflowExecutionHistory_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkflowExecutionHistoryRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkflowExecutionHistory_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetWorkflowExecutionHistory(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetWorkflowExecutionHistory_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkflowExecutionHistoryRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkflowExecutionHistory_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetWorkflowExecutionHistory(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_GetWorkflowExecutionHistory_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "execution": 1, "workflow_id": 2}, Base: []int{1, 1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 3, 2, 4}} +) + +func request_WorkflowService_GetWorkflowExecutionHistory_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkflowExecutionHistoryRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkflowExecutionHistory_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetWorkflowExecutionHistory(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetWorkflowExecutionHistory_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkflowExecutionHistoryRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkflowExecutionHistory_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetWorkflowExecutionHistory(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_GetWorkflowExecutionHistoryReverse_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "execution": 1, "workflow_id": 2}, Base: []int{1, 1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 3, 2, 4}} +) + +func request_WorkflowService_GetWorkflowExecutionHistoryReverse_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkflowExecutionHistoryReverseRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkflowExecutionHistoryReverse_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetWorkflowExecutionHistoryReverse(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetWorkflowExecutionHistoryReverse_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkflowExecutionHistoryReverseRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkflowExecutionHistoryReverse_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetWorkflowExecutionHistoryReverse(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_GetWorkflowExecutionHistoryReverse_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "execution": 1, "workflow_id": 2}, Base: []int{1, 1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 3, 2, 4}} +) + +func request_WorkflowService_GetWorkflowExecutionHistoryReverse_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkflowExecutionHistoryReverseRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkflowExecutionHistoryReverse_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetWorkflowExecutionHistoryReverse(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetWorkflowExecutionHistoryReverse_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkflowExecutionHistoryReverseRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkflowExecutionHistoryReverse_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetWorkflowExecutionHistoryReverse(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RecordActivityTaskHeartbeat_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RecordActivityTaskHeartbeatRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RecordActivityTaskHeartbeat(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RecordActivityTaskHeartbeat_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RecordActivityTaskHeartbeatRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RecordActivityTaskHeartbeat(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RecordActivityTaskHeartbeat_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RecordActivityTaskHeartbeatRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RecordActivityTaskHeartbeat(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RecordActivityTaskHeartbeat_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RecordActivityTaskHeartbeatRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RecordActivityTaskHeartbeat(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RecordActivityTaskHeartbeatById_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RecordActivityTaskHeartbeatByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RecordActivityTaskHeartbeatById(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RecordActivityTaskHeartbeatById_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RecordActivityTaskHeartbeatByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RecordActivityTaskHeartbeatById(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RecordActivityTaskHeartbeatById_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RecordActivityTaskHeartbeatByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RecordActivityTaskHeartbeatById(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RecordActivityTaskHeartbeatById_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RecordActivityTaskHeartbeatByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RecordActivityTaskHeartbeatById(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RespondActivityTaskCompleted_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCompletedRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RespondActivityTaskCompleted(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RespondActivityTaskCompleted_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCompletedRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RespondActivityTaskCompleted(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RespondActivityTaskCompleted_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCompletedRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RespondActivityTaskCompleted(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RespondActivityTaskCompleted_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCompletedRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RespondActivityTaskCompleted(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RespondActivityTaskCompletedById_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCompletedByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RespondActivityTaskCompletedById(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RespondActivityTaskCompletedById_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCompletedByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RespondActivityTaskCompletedById(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RespondActivityTaskCompletedById_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCompletedByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RespondActivityTaskCompletedById(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RespondActivityTaskCompletedById_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCompletedByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RespondActivityTaskCompletedById(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RespondActivityTaskFailed_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskFailedRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RespondActivityTaskFailed(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RespondActivityTaskFailed_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskFailedRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RespondActivityTaskFailed(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RespondActivityTaskFailed_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskFailedRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RespondActivityTaskFailed(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RespondActivityTaskFailed_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskFailedRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RespondActivityTaskFailed(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RespondActivityTaskFailedById_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskFailedByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RespondActivityTaskFailedById(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RespondActivityTaskFailedById_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskFailedByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RespondActivityTaskFailedById(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RespondActivityTaskFailedById_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskFailedByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RespondActivityTaskFailedById(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RespondActivityTaskFailedById_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskFailedByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RespondActivityTaskFailedById(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RespondActivityTaskCanceled_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCanceledRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RespondActivityTaskCanceled(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RespondActivityTaskCanceled_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCanceledRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RespondActivityTaskCanceled(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RespondActivityTaskCanceled_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCanceledRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RespondActivityTaskCanceled(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RespondActivityTaskCanceled_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCanceledRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RespondActivityTaskCanceled(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RespondActivityTaskCanceledById_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCanceledByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RespondActivityTaskCanceledById(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RespondActivityTaskCanceledById_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCanceledByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RespondActivityTaskCanceledById(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RespondActivityTaskCanceledById_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCanceledByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.RespondActivityTaskCanceledById(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RespondActivityTaskCanceledById_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RespondActivityTaskCanceledByIdRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.RespondActivityTaskCanceledById(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RequestCancelWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RequestCancelWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + msg, err := client.RequestCancelWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RequestCancelWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RequestCancelWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + msg, err := server.RequestCancelWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_RequestCancelWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RequestCancelWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + msg, err := client.RequestCancelWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_RequestCancelWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq RequestCancelWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + msg, err := server.RequestCancelWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_SignalWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq SignalWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + val, ok = pathParams["signal_name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "signal_name") + } + + protoReq.SignalName, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "signal_name", err) + } + + msg, err := client.SignalWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_SignalWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq SignalWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + val, ok = pathParams["signal_name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "signal_name") + } + + protoReq.SignalName, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "signal_name", err) + } + + msg, err := server.SignalWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_SignalWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq SignalWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + val, ok = pathParams["signal_name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "signal_name") + } + + protoReq.SignalName, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "signal_name", err) + } + + msg, err := client.SignalWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_SignalWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq SignalWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + val, ok = pathParams["signal_name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "signal_name") + } + + protoReq.SignalName, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "signal_name", err) + } + + msg, err := server.SignalWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_SignalWithStartWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq SignalWithStartWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_id") + } + + protoReq.WorkflowId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_id", err) + } + + val, ok = pathParams["signal_name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "signal_name") + } + + protoReq.SignalName, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "signal_name", err) + } + + msg, err := client.SignalWithStartWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_SignalWithStartWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq SignalWithStartWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_id") + } + + protoReq.WorkflowId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_id", err) + } + + val, ok = pathParams["signal_name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "signal_name") + } + + protoReq.SignalName, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "signal_name", err) + } + + msg, err := server.SignalWithStartWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_SignalWithStartWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq SignalWithStartWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_id") + } + + protoReq.WorkflowId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_id", err) + } + + val, ok = pathParams["signal_name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "signal_name") + } + + protoReq.SignalName, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "signal_name", err) + } + + msg, err := client.SignalWithStartWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_SignalWithStartWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq SignalWithStartWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_id") + } + + protoReq.WorkflowId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_id", err) + } + + val, ok = pathParams["signal_name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "signal_name") + } + + protoReq.SignalName, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "signal_name", err) + } + + msg, err := server.SignalWithStartWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_ResetWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ResetWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + msg, err := client.ResetWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ResetWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ResetWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + msg, err := server.ResetWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_ResetWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ResetWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + msg, err := client.ResetWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ResetWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ResetWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + msg, err := server.ResetWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_TerminateWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq TerminateWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + msg, err := client.TerminateWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_TerminateWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq TerminateWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + msg, err := server.TerminateWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_TerminateWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq TerminateWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + msg, err := client.TerminateWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_TerminateWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq TerminateWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + msg, err := server.TerminateWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_ListWorkflowExecutions_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_ListWorkflowExecutions_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListWorkflowExecutionsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListWorkflowExecutions_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListWorkflowExecutions(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ListWorkflowExecutions_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListWorkflowExecutionsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListWorkflowExecutions_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListWorkflowExecutions(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_ListWorkflowExecutions_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_ListWorkflowExecutions_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListWorkflowExecutionsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListWorkflowExecutions_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListWorkflowExecutions(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ListWorkflowExecutions_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListWorkflowExecutionsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListWorkflowExecutions_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListWorkflowExecutions(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_ListArchivedWorkflowExecutions_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_ListArchivedWorkflowExecutions_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListArchivedWorkflowExecutionsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListArchivedWorkflowExecutions_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListArchivedWorkflowExecutions(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ListArchivedWorkflowExecutions_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListArchivedWorkflowExecutionsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListArchivedWorkflowExecutions_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListArchivedWorkflowExecutions(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_ListArchivedWorkflowExecutions_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_ListArchivedWorkflowExecutions_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListArchivedWorkflowExecutionsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListArchivedWorkflowExecutions_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListArchivedWorkflowExecutions(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ListArchivedWorkflowExecutions_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListArchivedWorkflowExecutionsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListArchivedWorkflowExecutions_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListArchivedWorkflowExecutions(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_CountWorkflowExecutions_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_CountWorkflowExecutions_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CountWorkflowExecutionsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_CountWorkflowExecutions_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.CountWorkflowExecutions(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_CountWorkflowExecutions_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CountWorkflowExecutionsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_CountWorkflowExecutions_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.CountWorkflowExecutions(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_CountWorkflowExecutions_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_CountWorkflowExecutions_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CountWorkflowExecutionsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_CountWorkflowExecutions_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.CountWorkflowExecutions(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_CountWorkflowExecutions_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CountWorkflowExecutionsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_CountWorkflowExecutions_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.CountWorkflowExecutions(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_QueryWorkflow_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq QueryWorkflowRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + val, ok = pathParams["query.query_type"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "query.query_type") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "query.query_type", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "query.query_type", err) + } + + msg, err := client.QueryWorkflow(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_QueryWorkflow_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq QueryWorkflowRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + val, ok = pathParams["query.query_type"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "query.query_type") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "query.query_type", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "query.query_type", err) + } + + msg, err := server.QueryWorkflow(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_QueryWorkflow_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq QueryWorkflowRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + val, ok = pathParams["query.query_type"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "query.query_type") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "query.query_type", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "query.query_type", err) + } + + msg, err := client.QueryWorkflow(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_QueryWorkflow_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq QueryWorkflowRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + val, ok = pathParams["query.query_type"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "query.query_type") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "query.query_type", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "query.query_type", err) + } + + msg, err := server.QueryWorkflow(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_DescribeWorkflowExecution_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "execution": 1, "workflow_id": 2}, Base: []int{1, 1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 3, 2, 4}} +) + +func request_WorkflowService_DescribeWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DescribeWorkflowExecution_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DescribeWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_DescribeWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DescribeWorkflowExecution_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DescribeWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_DescribeWorkflowExecution_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "execution": 1, "workflow_id": 2}, Base: []int{1, 1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 3, 2, 4}} +) + +func request_WorkflowService_DescribeWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DescribeWorkflowExecution_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DescribeWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_DescribeWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "execution.workflow_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DescribeWorkflowExecution_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DescribeWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_DescribeTaskQueue_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "task_queue": 1, "name": 2}, Base: []int{1, 1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 3, 2, 4}} +) + +func request_WorkflowService_DescribeTaskQueue_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeTaskQueueRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["task_queue.name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "task_queue.name") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "task_queue.name", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "task_queue.name", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DescribeTaskQueue_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DescribeTaskQueue(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_DescribeTaskQueue_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeTaskQueueRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["task_queue.name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "task_queue.name") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "task_queue.name", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "task_queue.name", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DescribeTaskQueue_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DescribeTaskQueue(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_DescribeTaskQueue_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "task_queue": 1, "name": 2}, Base: []int{1, 1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 3, 2, 4}} +) + +func request_WorkflowService_DescribeTaskQueue_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeTaskQueueRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["task_queue.name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "task_queue.name") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "task_queue.name", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "task_queue.name", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DescribeTaskQueue_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DescribeTaskQueue(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_DescribeTaskQueue_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeTaskQueueRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["task_queue.name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "task_queue.name") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "task_queue.name", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "task_queue.name", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DescribeTaskQueue_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DescribeTaskQueue(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_GetClusterInfo_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetClusterInfoRequest + var metadata runtime.ServerMetadata + + msg, err := client.GetClusterInfo(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetClusterInfo_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetClusterInfoRequest + var metadata runtime.ServerMetadata + + msg, err := server.GetClusterInfo(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_GetClusterInfo_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetClusterInfoRequest + var metadata runtime.ServerMetadata + + msg, err := client.GetClusterInfo(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetClusterInfo_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetClusterInfoRequest + var metadata runtime.ServerMetadata + + msg, err := server.GetClusterInfo(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_GetSystemInfo_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetSystemInfoRequest + var metadata runtime.ServerMetadata + + msg, err := client.GetSystemInfo(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetSystemInfo_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetSystemInfoRequest + var metadata runtime.ServerMetadata + + msg, err := server.GetSystemInfo(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_GetSystemInfo_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetSystemInfoRequest + var metadata runtime.ServerMetadata + + msg, err := client.GetSystemInfo(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetSystemInfo_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetSystemInfoRequest + var metadata runtime.ServerMetadata + + msg, err := server.GetSystemInfo(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_CreateSchedule_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateScheduleRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := client.CreateSchedule(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_CreateSchedule_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateScheduleRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := server.CreateSchedule(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_CreateSchedule_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateScheduleRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := client.CreateSchedule(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_CreateSchedule_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateScheduleRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := server.CreateSchedule(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_DescribeSchedule_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeScheduleRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := client.DescribeSchedule(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_DescribeSchedule_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeScheduleRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := server.DescribeSchedule(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_DescribeSchedule_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeScheduleRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := client.DescribeSchedule(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_DescribeSchedule_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeScheduleRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := server.DescribeSchedule(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_UpdateSchedule_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateScheduleRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := client.UpdateSchedule(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_UpdateSchedule_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateScheduleRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := server.UpdateSchedule(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_UpdateSchedule_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateScheduleRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := client.UpdateSchedule(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_UpdateSchedule_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateScheduleRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := server.UpdateSchedule(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_PatchSchedule_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq PatchScheduleRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := client.PatchSchedule(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_PatchSchedule_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq PatchScheduleRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := server.PatchSchedule(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_PatchSchedule_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq PatchScheduleRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := client.PatchSchedule(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_PatchSchedule_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq PatchScheduleRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + msg, err := server.PatchSchedule(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_ListScheduleMatchingTimes_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "schedule_id": 1}, Base: []int{1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 2, 3}} +) + +func request_WorkflowService_ListScheduleMatchingTimes_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListScheduleMatchingTimesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListScheduleMatchingTimes_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListScheduleMatchingTimes(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ListScheduleMatchingTimes_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListScheduleMatchingTimesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListScheduleMatchingTimes_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListScheduleMatchingTimes(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_ListScheduleMatchingTimes_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "schedule_id": 1}, Base: []int{1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 2, 3}} +) + +func request_WorkflowService_ListScheduleMatchingTimes_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListScheduleMatchingTimesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListScheduleMatchingTimes_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListScheduleMatchingTimes(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ListScheduleMatchingTimes_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListScheduleMatchingTimesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListScheduleMatchingTimes_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListScheduleMatchingTimes(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_DeleteSchedule_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "schedule_id": 1}, Base: []int{1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 2, 3}} +) + +func request_WorkflowService_DeleteSchedule_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteScheduleRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DeleteSchedule_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DeleteSchedule(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_DeleteSchedule_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteScheduleRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DeleteSchedule_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DeleteSchedule(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_DeleteSchedule_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "schedule_id": 1}, Base: []int{1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 2, 3}} +) + +func request_WorkflowService_DeleteSchedule_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteScheduleRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DeleteSchedule_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.DeleteSchedule(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_DeleteSchedule_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteScheduleRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["schedule_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "schedule_id") + } + + protoReq.ScheduleId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "schedule_id", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_DeleteSchedule_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.DeleteSchedule(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_ListSchedules_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_ListSchedules_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListSchedulesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListSchedules_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListSchedules(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ListSchedules_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListSchedulesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListSchedules_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListSchedules(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_ListSchedules_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_ListSchedules_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListSchedulesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListSchedules_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListSchedules(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ListSchedules_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListSchedulesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListSchedules_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListSchedules(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_GetWorkerBuildIdCompatibility_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "task_queue": 1}, Base: []int{1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 2, 3}} +) + +func request_WorkflowService_GetWorkerBuildIdCompatibility_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkerBuildIdCompatibilityRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["task_queue"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "task_queue") + } + + protoReq.TaskQueue, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "task_queue", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkerBuildIdCompatibility_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetWorkerBuildIdCompatibility(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetWorkerBuildIdCompatibility_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkerBuildIdCompatibilityRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["task_queue"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "task_queue") + } + + protoReq.TaskQueue, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "task_queue", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkerBuildIdCompatibility_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetWorkerBuildIdCompatibility(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_GetWorkerBuildIdCompatibility_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0, "task_queue": 1}, Base: []int{1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 2, 3}} +) + +func request_WorkflowService_GetWorkerBuildIdCompatibility_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkerBuildIdCompatibilityRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["task_queue"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "task_queue") + } + + protoReq.TaskQueue, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "task_queue", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkerBuildIdCompatibility_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetWorkerBuildIdCompatibility(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetWorkerBuildIdCompatibility_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkerBuildIdCompatibilityRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["task_queue"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "task_queue") + } + + protoReq.TaskQueue, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "task_queue", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkerBuildIdCompatibility_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetWorkerBuildIdCompatibility(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_GetWorkerVersioningRules_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkerVersioningRulesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["task_queue"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "task_queue") + } + + protoReq.TaskQueue, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "task_queue", err) + } + + msg, err := client.GetWorkerVersioningRules(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetWorkerVersioningRules_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkerVersioningRulesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["task_queue"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "task_queue") + } + + protoReq.TaskQueue, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "task_queue", err) + } + + msg, err := server.GetWorkerVersioningRules(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_GetWorkerVersioningRules_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkerVersioningRulesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["task_queue"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "task_queue") + } + + protoReq.TaskQueue, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "task_queue", err) + } + + msg, err := client.GetWorkerVersioningRules(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetWorkerVersioningRules_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkerVersioningRulesRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["task_queue"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "task_queue") + } + + protoReq.TaskQueue, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "task_queue", err) + } + + msg, err := server.GetWorkerVersioningRules(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_GetWorkerTaskReachability_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_GetWorkerTaskReachability_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkerTaskReachabilityRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkerTaskReachability_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetWorkerTaskReachability(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetWorkerTaskReachability_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkerTaskReachabilityRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkerTaskReachability_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetWorkerTaskReachability(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_GetWorkerTaskReachability_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_GetWorkerTaskReachability_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkerTaskReachabilityRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkerTaskReachability_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.GetWorkerTaskReachability(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_GetWorkerTaskReachability_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetWorkerTaskReachabilityRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_GetWorkerTaskReachability_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.GetWorkerTaskReachability(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_UpdateWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + val, ok = pathParams["request.input.name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "request.input.name") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "request.input.name", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "request.input.name", err) + } + + msg, err := client.UpdateWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_UpdateWorkflowExecution_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + val, ok = pathParams["request.input.name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "request.input.name") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "request.input.name", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "request.input.name", err) + } + + msg, err := server.UpdateWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_UpdateWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + val, ok = pathParams["request.input.name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "request.input.name") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "request.input.name", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "request.input.name", err) + } + + msg, err := client.UpdateWorkflowExecution(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_UpdateWorkflowExecution_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq UpdateWorkflowExecutionRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["workflow_execution.workflow_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "workflow_execution.workflow_id") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "workflow_execution.workflow_id", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "workflow_execution.workflow_id", err) + } + + val, ok = pathParams["request.input.name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "request.input.name") + } + + err = runtime.PopulateFieldFromPath(&protoReq, "request.input.name", val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "request.input.name", err) + } + + msg, err := server.UpdateWorkflowExecution(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_StartBatchOperation_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq StartBatchOperationRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["job_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "job_id") + } + + protoReq.JobId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "job_id", err) + } + + msg, err := client.StartBatchOperation(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_StartBatchOperation_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq StartBatchOperationRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["job_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "job_id") + } + + protoReq.JobId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "job_id", err) + } + + msg, err := server.StartBatchOperation(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_StartBatchOperation_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq StartBatchOperationRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["job_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "job_id") + } + + protoReq.JobId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "job_id", err) + } + + msg, err := client.StartBatchOperation(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_StartBatchOperation_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq StartBatchOperationRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["job_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "job_id") + } + + protoReq.JobId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "job_id", err) + } + + msg, err := server.StartBatchOperation(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_StopBatchOperation_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq StopBatchOperationRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["job_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "job_id") + } + + protoReq.JobId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "job_id", err) + } + + msg, err := client.StopBatchOperation(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_StopBatchOperation_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq StopBatchOperationRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["job_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "job_id") + } + + protoReq.JobId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "job_id", err) + } + + msg, err := server.StopBatchOperation(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_StopBatchOperation_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq StopBatchOperationRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["job_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "job_id") + } + + protoReq.JobId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "job_id", err) + } + + msg, err := client.StopBatchOperation(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_StopBatchOperation_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq StopBatchOperationRequest + var metadata runtime.ServerMetadata + + if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["job_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "job_id") + } + + protoReq.JobId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "job_id", err) + } + + msg, err := server.StopBatchOperation(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_DescribeBatchOperation_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeBatchOperationRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["job_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "job_id") + } + + protoReq.JobId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "job_id", err) + } + + msg, err := client.DescribeBatchOperation(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_DescribeBatchOperation_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeBatchOperationRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["job_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "job_id") + } + + protoReq.JobId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "job_id", err) + } + + msg, err := server.DescribeBatchOperation(ctx, &protoReq) + return msg, metadata, err + +} + +func request_WorkflowService_DescribeBatchOperation_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeBatchOperationRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["job_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "job_id") + } + + protoReq.JobId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "job_id", err) + } + + msg, err := client.DescribeBatchOperation(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_DescribeBatchOperation_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DescribeBatchOperationRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["job_id"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "job_id") + } + + protoReq.JobId, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "job_id", err) + } + + msg, err := server.DescribeBatchOperation(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_ListBatchOperations_0 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_ListBatchOperations_0(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListBatchOperationsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListBatchOperations_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListBatchOperations(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ListBatchOperations_0(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListBatchOperationsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListBatchOperations_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListBatchOperations(ctx, &protoReq) + return msg, metadata, err + +} + +var ( + filter_WorkflowService_ListBatchOperations_1 = &utilities.DoubleArray{Encoding: map[string]int{"namespace": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_WorkflowService_ListBatchOperations_1(ctx context.Context, marshaler runtime.Marshaler, client WorkflowServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListBatchOperationsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListBatchOperations_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.ListBatchOperations(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_WorkflowService_ListBatchOperations_1(ctx context.Context, marshaler runtime.Marshaler, server WorkflowServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListBatchOperationsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_WorkflowService_ListBatchOperations_1); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.ListBatchOperations(ctx, &protoReq) + return msg, metadata, err + +} + +// RegisterWorkflowServiceHandlerServer registers the http handlers for service WorkflowService to "mux". +// UnaryRPC :call WorkflowServiceServer directly. +// StreamingRPC :currently unsupported pending https://github.com/grpc/grpc-go/issues/906. +// Note that using this registration option will cause many gRPC library features to stop working. Consider using RegisterWorkflowServiceHandlerFromEndpoint instead. +// GRPC interceptors will not work for this type of registration. To use interceptors, you must use the "runtime.WithMiddlewares" option in the "runtime.NewServeMux" call. +func RegisterWorkflowServiceHandlerServer(ctx context.Context, mux *runtime.ServeMux, server WorkflowServiceServer) error { + + mux.Handle("POST", pattern_WorkflowService_RegisterNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RegisterNamespace", runtime.WithHTTPPathPattern("/cluster/namespaces")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RegisterNamespace_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RegisterNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RegisterNamespace_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RegisterNamespace", runtime.WithHTTPPathPattern("/api/v1/namespaces")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RegisterNamespace_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RegisterNamespace_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeNamespace", runtime.WithHTTPPathPattern("/cluster/namespaces/{namespace}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_DescribeNamespace_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeNamespace_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeNamespace", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_DescribeNamespace_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeNamespace_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListNamespaces_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListNamespaces", runtime.WithHTTPPathPattern("/cluster/namespaces")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ListNamespaces_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListNamespaces_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListNamespaces_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListNamespaces", runtime.WithHTTPPathPattern("/api/v1/namespaces")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ListNamespaces_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListNamespaces_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_UpdateNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/UpdateNamespace", runtime.WithHTTPPathPattern("/cluster/namespaces/{namespace}/update")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_UpdateNamespace_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_UpdateNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_UpdateNamespace_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/UpdateNamespace", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/update")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_UpdateNamespace_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_UpdateNamespace_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_StartWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/StartWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_StartWorkflowExecution_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_StartWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_StartWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/StartWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_StartWorkflowExecution_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_StartWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_ExecuteMultiOperation_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ExecuteMultiOperation", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/execute-multi-operation")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ExecuteMultiOperation_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ExecuteMultiOperation_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_ExecuteMultiOperation_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ExecuteMultiOperation", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/execute-multi-operation")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ExecuteMultiOperation_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ExecuteMultiOperation_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkflowExecutionHistory_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkflowExecutionHistory", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{execution.workflow_id}/history")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetWorkflowExecutionHistory_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkflowExecutionHistory_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkflowExecutionHistory_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkflowExecutionHistory", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/history")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetWorkflowExecutionHistory_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkflowExecutionHistory_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkflowExecutionHistoryReverse_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkflowExecutionHistoryReverse", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{execution.workflow_id}/history-reverse")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetWorkflowExecutionHistoryReverse_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkflowExecutionHistoryReverse_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkflowExecutionHistoryReverse_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkflowExecutionHistoryReverse", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/history-reverse")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetWorkflowExecutionHistoryReverse_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkflowExecutionHistoryReverse_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RecordActivityTaskHeartbeat_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RecordActivityTaskHeartbeat", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/heartbeat")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RecordActivityTaskHeartbeat_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RecordActivityTaskHeartbeat_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RecordActivityTaskHeartbeat_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RecordActivityTaskHeartbeat", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/heartbeat")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RecordActivityTaskHeartbeat_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RecordActivityTaskHeartbeat_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RecordActivityTaskHeartbeatById_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RecordActivityTaskHeartbeatById", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/heartbeat-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RecordActivityTaskHeartbeatById_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RecordActivityTaskHeartbeatById_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RecordActivityTaskHeartbeatById_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RecordActivityTaskHeartbeatById", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/heartbeat-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RecordActivityTaskHeartbeatById_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RecordActivityTaskHeartbeatById_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCompleted_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCompleted", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/complete")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RespondActivityTaskCompleted_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCompleted_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCompleted_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCompleted", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/complete")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RespondActivityTaskCompleted_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCompleted_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCompletedById_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCompletedById", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/complete-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RespondActivityTaskCompletedById_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCompletedById_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCompletedById_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCompletedById", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/complete-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RespondActivityTaskCompletedById_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCompletedById_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskFailed_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskFailed", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/fail")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RespondActivityTaskFailed_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskFailed_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskFailed_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskFailed", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/fail")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RespondActivityTaskFailed_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskFailed_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskFailedById_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskFailedById", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/fail-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RespondActivityTaskFailedById_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskFailedById_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskFailedById_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskFailedById", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/fail-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RespondActivityTaskFailedById_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskFailedById_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCanceled_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCanceled", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/cancel")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RespondActivityTaskCanceled_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCanceled_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCanceled_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCanceled", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/cancel")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RespondActivityTaskCanceled_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCanceled_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCanceledById_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCanceledById", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/cancel-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RespondActivityTaskCanceledById_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCanceledById_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCanceledById_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCanceledById", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/cancel-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RespondActivityTaskCanceledById_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCanceledById_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RequestCancelWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RequestCancelWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/cancel")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RequestCancelWorkflowExecution_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RequestCancelWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RequestCancelWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RequestCancelWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/cancel")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_RequestCancelWorkflowExecution_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RequestCancelWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_SignalWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/SignalWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/signal/{signal_name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_SignalWorkflowExecution_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_SignalWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_SignalWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/SignalWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/signal/{signal_name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_SignalWorkflowExecution_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_SignalWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_SignalWithStartWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/SignalWithStartWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_id}/signal-with-start/{signal_name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_SignalWithStartWorkflowExecution_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_SignalWithStartWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_SignalWithStartWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/SignalWithStartWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_id}/signal-with-start/{signal_name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_SignalWithStartWorkflowExecution_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_SignalWithStartWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_ResetWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ResetWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/reset")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ResetWorkflowExecution_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ResetWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_ResetWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ResetWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/reset")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ResetWorkflowExecution_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ResetWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_TerminateWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/TerminateWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/terminate")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_TerminateWorkflowExecution_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_TerminateWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_TerminateWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/TerminateWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/terminate")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_TerminateWorkflowExecution_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_TerminateWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListWorkflowExecutions_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListWorkflowExecutions", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ListWorkflowExecutions_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListWorkflowExecutions_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListWorkflowExecutions_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListWorkflowExecutions", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ListWorkflowExecutions_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListWorkflowExecutions_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListArchivedWorkflowExecutions_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListArchivedWorkflowExecutions", runtime.WithHTTPPathPattern("/namespaces/{namespace}/archived-workflows")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ListArchivedWorkflowExecutions_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListArchivedWorkflowExecutions_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListArchivedWorkflowExecutions_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListArchivedWorkflowExecutions", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/archived-workflows")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ListArchivedWorkflowExecutions_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListArchivedWorkflowExecutions_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_CountWorkflowExecutions_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/CountWorkflowExecutions", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflow-count")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_CountWorkflowExecutions_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_CountWorkflowExecutions_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_CountWorkflowExecutions_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/CountWorkflowExecutions", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflow-count")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_CountWorkflowExecutions_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_CountWorkflowExecutions_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_QueryWorkflow_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/QueryWorkflow", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{execution.workflow_id}/query/{query.query_type}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_QueryWorkflow_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_QueryWorkflow_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_QueryWorkflow_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/QueryWorkflow", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/query/{query.query_type}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_QueryWorkflow_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_QueryWorkflow_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{execution.workflow_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_DescribeWorkflowExecution_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_DescribeWorkflowExecution_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeTaskQueue_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeTaskQueue", runtime.WithHTTPPathPattern("/namespaces/{namespace}/task-queues/{task_queue.name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_DescribeTaskQueue_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeTaskQueue_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeTaskQueue_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeTaskQueue", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/task-queues/{task_queue.name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_DescribeTaskQueue_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeTaskQueue_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetClusterInfo_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetClusterInfo", runtime.WithHTTPPathPattern("/cluster")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetClusterInfo_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetClusterInfo_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetClusterInfo_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetClusterInfo", runtime.WithHTTPPathPattern("/api/v1/cluster-info")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetClusterInfo_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetClusterInfo_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetSystemInfo_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetSystemInfo", runtime.WithHTTPPathPattern("/system-info")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetSystemInfo_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetSystemInfo_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetSystemInfo_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetSystemInfo", runtime.WithHTTPPathPattern("/api/v1/system-info")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetSystemInfo_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetSystemInfo_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_CreateSchedule_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/CreateSchedule", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules/{schedule_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_CreateSchedule_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_CreateSchedule_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_CreateSchedule_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/CreateSchedule", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules/{schedule_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_CreateSchedule_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_CreateSchedule_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeSchedule_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeSchedule", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules/{schedule_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_DescribeSchedule_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeSchedule_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeSchedule_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeSchedule", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules/{schedule_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_DescribeSchedule_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeSchedule_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_UpdateSchedule_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/UpdateSchedule", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules/{schedule_id}/update")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_UpdateSchedule_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_UpdateSchedule_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_UpdateSchedule_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/UpdateSchedule", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules/{schedule_id}/update")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_UpdateSchedule_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_UpdateSchedule_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_PatchSchedule_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/PatchSchedule", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules/{schedule_id}/patch")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_PatchSchedule_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_PatchSchedule_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_PatchSchedule_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/PatchSchedule", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules/{schedule_id}/patch")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_PatchSchedule_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_PatchSchedule_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListScheduleMatchingTimes_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListScheduleMatchingTimes", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules/{schedule_id}/matching-times")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ListScheduleMatchingTimes_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListScheduleMatchingTimes_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListScheduleMatchingTimes_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListScheduleMatchingTimes", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules/{schedule_id}/matching-times")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ListScheduleMatchingTimes_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListScheduleMatchingTimes_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_WorkflowService_DeleteSchedule_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DeleteSchedule", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules/{schedule_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_DeleteSchedule_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DeleteSchedule_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_WorkflowService_DeleteSchedule_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DeleteSchedule", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules/{schedule_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_DeleteSchedule_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DeleteSchedule_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListSchedules_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListSchedules", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ListSchedules_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListSchedules_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListSchedules_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListSchedules", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ListSchedules_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListSchedules_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkerBuildIdCompatibility_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerBuildIdCompatibility", runtime.WithHTTPPathPattern("/namespaces/{namespace}/task-queues/{task_queue}/worker-build-id-compatibility")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetWorkerBuildIdCompatibility_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkerBuildIdCompatibility_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkerBuildIdCompatibility_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerBuildIdCompatibility", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/task-queues/{task_queue}/worker-build-id-compatibility")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetWorkerBuildIdCompatibility_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkerBuildIdCompatibility_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkerVersioningRules_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerVersioningRules", runtime.WithHTTPPathPattern("/namespaces/{namespace}/task-queues/{task_queue}/worker-versioning-rules")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetWorkerVersioningRules_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkerVersioningRules_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkerVersioningRules_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerVersioningRules", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/task-queues/{task_queue}/worker-versioning-rules")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetWorkerVersioningRules_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkerVersioningRules_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkerTaskReachability_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerTaskReachability", runtime.WithHTTPPathPattern("/namespaces/{namespace}/worker-task-reachability")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetWorkerTaskReachability_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkerTaskReachability_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkerTaskReachability_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerTaskReachability", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/worker-task-reachability")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_GetWorkerTaskReachability_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkerTaskReachability_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_UpdateWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/UpdateWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update/{request.input.name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_UpdateWorkflowExecution_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_UpdateWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_UpdateWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/UpdateWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update/{request.input.name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_UpdateWorkflowExecution_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_UpdateWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_StartBatchOperation_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/StartBatchOperation", runtime.WithHTTPPathPattern("/namespaces/{namespace}/batch-operations/{job_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_StartBatchOperation_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_StartBatchOperation_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_StartBatchOperation_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/StartBatchOperation", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/batch-operations/{job_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_StartBatchOperation_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_StartBatchOperation_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_StopBatchOperation_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/StopBatchOperation", runtime.WithHTTPPathPattern("/namespaces/{namespace}/batch-operations/{job_id}/stop")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_StopBatchOperation_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_StopBatchOperation_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_StopBatchOperation_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/StopBatchOperation", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/batch-operations/{job_id}/stop")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_StopBatchOperation_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_StopBatchOperation_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeBatchOperation_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeBatchOperation", runtime.WithHTTPPathPattern("/namespaces/{namespace}/batch-operations/{job_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_DescribeBatchOperation_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeBatchOperation_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeBatchOperation_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeBatchOperation", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/batch-operations/{job_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_DescribeBatchOperation_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeBatchOperation_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListBatchOperations_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListBatchOperations", runtime.WithHTTPPathPattern("/namespaces/{namespace}/batch-operations")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ListBatchOperations_0(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListBatchOperations_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListBatchOperations_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateIncomingContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListBatchOperations", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/batch-operations")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_WorkflowService_ListBatchOperations_1(annotatedContext, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListBatchOperations_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + return nil +} + +// RegisterWorkflowServiceHandlerFromEndpoint is same as RegisterWorkflowServiceHandler but +// automatically dials to "endpoint" and closes the connection when "ctx" gets done. +func RegisterWorkflowServiceHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) { + conn, err := grpc.NewClient(endpoint, opts...) + if err != nil { + return err + } + defer func() { + if err != nil { + if cerr := conn.Close(); cerr != nil { + grpclog.Errorf("Failed to close conn to %s: %v", endpoint, cerr) + } + return + } + go func() { + <-ctx.Done() + if cerr := conn.Close(); cerr != nil { + grpclog.Errorf("Failed to close conn to %s: %v", endpoint, cerr) + } + }() + }() + + return RegisterWorkflowServiceHandler(ctx, mux, conn) +} + +// RegisterWorkflowServiceHandler registers the http handlers for service WorkflowService to "mux". +// The handlers forward requests to the grpc endpoint over "conn". +func RegisterWorkflowServiceHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error { + return RegisterWorkflowServiceHandlerClient(ctx, mux, NewWorkflowServiceClient(conn)) +} + +// RegisterWorkflowServiceHandlerClient registers the http handlers for service WorkflowService +// to "mux". The handlers forward requests to the grpc endpoint over the given implementation of "WorkflowServiceClient". +// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "WorkflowServiceClient" +// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in +// "WorkflowServiceClient" to call the correct interceptors. This client ignores the HTTP middlewares. +func RegisterWorkflowServiceHandlerClient(ctx context.Context, mux *runtime.ServeMux, client WorkflowServiceClient) error { + + mux.Handle("POST", pattern_WorkflowService_RegisterNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RegisterNamespace", runtime.WithHTTPPathPattern("/cluster/namespaces")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RegisterNamespace_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RegisterNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RegisterNamespace_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RegisterNamespace", runtime.WithHTTPPathPattern("/api/v1/namespaces")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RegisterNamespace_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RegisterNamespace_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeNamespace", runtime.WithHTTPPathPattern("/cluster/namespaces/{namespace}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_DescribeNamespace_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeNamespace_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeNamespace", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_DescribeNamespace_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeNamespace_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListNamespaces_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListNamespaces", runtime.WithHTTPPathPattern("/cluster/namespaces")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ListNamespaces_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListNamespaces_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListNamespaces_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListNamespaces", runtime.WithHTTPPathPattern("/api/v1/namespaces")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ListNamespaces_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListNamespaces_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_UpdateNamespace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/UpdateNamespace", runtime.WithHTTPPathPattern("/cluster/namespaces/{namespace}/update")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_UpdateNamespace_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_UpdateNamespace_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_UpdateNamespace_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/UpdateNamespace", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/update")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_UpdateNamespace_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_UpdateNamespace_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_StartWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/StartWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_StartWorkflowExecution_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_StartWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_StartWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/StartWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_StartWorkflowExecution_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_StartWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_ExecuteMultiOperation_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ExecuteMultiOperation", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/execute-multi-operation")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ExecuteMultiOperation_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ExecuteMultiOperation_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_ExecuteMultiOperation_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ExecuteMultiOperation", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/execute-multi-operation")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ExecuteMultiOperation_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ExecuteMultiOperation_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkflowExecutionHistory_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkflowExecutionHistory", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{execution.workflow_id}/history")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetWorkflowExecutionHistory_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkflowExecutionHistory_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkflowExecutionHistory_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkflowExecutionHistory", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/history")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetWorkflowExecutionHistory_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkflowExecutionHistory_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkflowExecutionHistoryReverse_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkflowExecutionHistoryReverse", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{execution.workflow_id}/history-reverse")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetWorkflowExecutionHistoryReverse_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkflowExecutionHistoryReverse_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkflowExecutionHistoryReverse_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkflowExecutionHistoryReverse", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/history-reverse")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetWorkflowExecutionHistoryReverse_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkflowExecutionHistoryReverse_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RecordActivityTaskHeartbeat_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RecordActivityTaskHeartbeat", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/heartbeat")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RecordActivityTaskHeartbeat_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RecordActivityTaskHeartbeat_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RecordActivityTaskHeartbeat_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RecordActivityTaskHeartbeat", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/heartbeat")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RecordActivityTaskHeartbeat_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RecordActivityTaskHeartbeat_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RecordActivityTaskHeartbeatById_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RecordActivityTaskHeartbeatById", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/heartbeat-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RecordActivityTaskHeartbeatById_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RecordActivityTaskHeartbeatById_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RecordActivityTaskHeartbeatById_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RecordActivityTaskHeartbeatById", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/heartbeat-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RecordActivityTaskHeartbeatById_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RecordActivityTaskHeartbeatById_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCompleted_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCompleted", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/complete")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RespondActivityTaskCompleted_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCompleted_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCompleted_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCompleted", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/complete")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RespondActivityTaskCompleted_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCompleted_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCompletedById_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCompletedById", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/complete-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RespondActivityTaskCompletedById_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCompletedById_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCompletedById_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCompletedById", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/complete-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RespondActivityTaskCompletedById_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCompletedById_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskFailed_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskFailed", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/fail")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RespondActivityTaskFailed_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskFailed_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskFailed_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskFailed", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/fail")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RespondActivityTaskFailed_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskFailed_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskFailedById_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskFailedById", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/fail-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RespondActivityTaskFailedById_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskFailedById_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskFailedById_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskFailedById", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/fail-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RespondActivityTaskFailedById_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskFailedById_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCanceled_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCanceled", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/cancel")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RespondActivityTaskCanceled_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCanceled_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCanceled_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCanceled", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/cancel")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RespondActivityTaskCanceled_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCanceled_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCanceledById_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCanceledById", runtime.WithHTTPPathPattern("/namespaces/{namespace}/activities/cancel-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RespondActivityTaskCanceledById_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCanceledById_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RespondActivityTaskCanceledById_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCanceledById", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/activities/cancel-by-id")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RespondActivityTaskCanceledById_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RespondActivityTaskCanceledById_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RequestCancelWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RequestCancelWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/cancel")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RequestCancelWorkflowExecution_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RequestCancelWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_RequestCancelWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/RequestCancelWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/cancel")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_RequestCancelWorkflowExecution_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_RequestCancelWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_SignalWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/SignalWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/signal/{signal_name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_SignalWorkflowExecution_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_SignalWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_SignalWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/SignalWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/signal/{signal_name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_SignalWorkflowExecution_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_SignalWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_SignalWithStartWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/SignalWithStartWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_id}/signal-with-start/{signal_name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_SignalWithStartWorkflowExecution_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_SignalWithStartWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_SignalWithStartWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/SignalWithStartWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_id}/signal-with-start/{signal_name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_SignalWithStartWorkflowExecution_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_SignalWithStartWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_ResetWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ResetWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/reset")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ResetWorkflowExecution_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ResetWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_ResetWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ResetWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/reset")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ResetWorkflowExecution_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ResetWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_TerminateWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/TerminateWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/terminate")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_TerminateWorkflowExecution_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_TerminateWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_TerminateWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/TerminateWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/terminate")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_TerminateWorkflowExecution_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_TerminateWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListWorkflowExecutions_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListWorkflowExecutions", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ListWorkflowExecutions_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListWorkflowExecutions_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListWorkflowExecutions_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListWorkflowExecutions", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ListWorkflowExecutions_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListWorkflowExecutions_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListArchivedWorkflowExecutions_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListArchivedWorkflowExecutions", runtime.WithHTTPPathPattern("/namespaces/{namespace}/archived-workflows")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ListArchivedWorkflowExecutions_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListArchivedWorkflowExecutions_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListArchivedWorkflowExecutions_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListArchivedWorkflowExecutions", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/archived-workflows")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ListArchivedWorkflowExecutions_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListArchivedWorkflowExecutions_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_CountWorkflowExecutions_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/CountWorkflowExecutions", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflow-count")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_CountWorkflowExecutions_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_CountWorkflowExecutions_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_CountWorkflowExecutions_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/CountWorkflowExecutions", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflow-count")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_CountWorkflowExecutions_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_CountWorkflowExecutions_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_QueryWorkflow_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/QueryWorkflow", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{execution.workflow_id}/query/{query.query_type}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_QueryWorkflow_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_QueryWorkflow_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_QueryWorkflow_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/QueryWorkflow", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}/query/{query.query_type}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_QueryWorkflow_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_QueryWorkflow_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{execution.workflow_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_DescribeWorkflowExecution_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{execution.workflow_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_DescribeWorkflowExecution_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeTaskQueue_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeTaskQueue", runtime.WithHTTPPathPattern("/namespaces/{namespace}/task-queues/{task_queue.name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_DescribeTaskQueue_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeTaskQueue_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeTaskQueue_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeTaskQueue", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/task-queues/{task_queue.name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_DescribeTaskQueue_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeTaskQueue_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetClusterInfo_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetClusterInfo", runtime.WithHTTPPathPattern("/cluster")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetClusterInfo_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetClusterInfo_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetClusterInfo_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetClusterInfo", runtime.WithHTTPPathPattern("/api/v1/cluster-info")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetClusterInfo_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetClusterInfo_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetSystemInfo_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetSystemInfo", runtime.WithHTTPPathPattern("/system-info")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetSystemInfo_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetSystemInfo_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetSystemInfo_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetSystemInfo", runtime.WithHTTPPathPattern("/api/v1/system-info")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetSystemInfo_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetSystemInfo_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_CreateSchedule_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/CreateSchedule", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules/{schedule_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_CreateSchedule_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_CreateSchedule_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_CreateSchedule_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/CreateSchedule", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules/{schedule_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_CreateSchedule_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_CreateSchedule_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeSchedule_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeSchedule", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules/{schedule_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_DescribeSchedule_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeSchedule_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeSchedule_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeSchedule", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules/{schedule_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_DescribeSchedule_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeSchedule_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_UpdateSchedule_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/UpdateSchedule", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules/{schedule_id}/update")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_UpdateSchedule_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_UpdateSchedule_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_UpdateSchedule_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/UpdateSchedule", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules/{schedule_id}/update")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_UpdateSchedule_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_UpdateSchedule_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_PatchSchedule_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/PatchSchedule", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules/{schedule_id}/patch")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_PatchSchedule_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_PatchSchedule_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_PatchSchedule_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/PatchSchedule", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules/{schedule_id}/patch")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_PatchSchedule_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_PatchSchedule_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListScheduleMatchingTimes_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListScheduleMatchingTimes", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules/{schedule_id}/matching-times")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ListScheduleMatchingTimes_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListScheduleMatchingTimes_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListScheduleMatchingTimes_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListScheduleMatchingTimes", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules/{schedule_id}/matching-times")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ListScheduleMatchingTimes_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListScheduleMatchingTimes_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_WorkflowService_DeleteSchedule_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DeleteSchedule", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules/{schedule_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_DeleteSchedule_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DeleteSchedule_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_WorkflowService_DeleteSchedule_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DeleteSchedule", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules/{schedule_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_DeleteSchedule_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DeleteSchedule_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListSchedules_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListSchedules", runtime.WithHTTPPathPattern("/namespaces/{namespace}/schedules")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ListSchedules_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListSchedules_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListSchedules_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListSchedules", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/schedules")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ListSchedules_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListSchedules_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkerBuildIdCompatibility_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerBuildIdCompatibility", runtime.WithHTTPPathPattern("/namespaces/{namespace}/task-queues/{task_queue}/worker-build-id-compatibility")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetWorkerBuildIdCompatibility_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkerBuildIdCompatibility_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkerBuildIdCompatibility_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerBuildIdCompatibility", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/task-queues/{task_queue}/worker-build-id-compatibility")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetWorkerBuildIdCompatibility_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkerBuildIdCompatibility_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkerVersioningRules_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerVersioningRules", runtime.WithHTTPPathPattern("/namespaces/{namespace}/task-queues/{task_queue}/worker-versioning-rules")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetWorkerVersioningRules_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkerVersioningRules_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkerVersioningRules_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerVersioningRules", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/task-queues/{task_queue}/worker-versioning-rules")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetWorkerVersioningRules_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkerVersioningRules_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkerTaskReachability_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerTaskReachability", runtime.WithHTTPPathPattern("/namespaces/{namespace}/worker-task-reachability")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetWorkerTaskReachability_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkerTaskReachability_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_GetWorkerTaskReachability_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerTaskReachability", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/worker-task-reachability")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_GetWorkerTaskReachability_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_GetWorkerTaskReachability_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_UpdateWorkflowExecution_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/UpdateWorkflowExecution", runtime.WithHTTPPathPattern("/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update/{request.input.name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_UpdateWorkflowExecution_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_UpdateWorkflowExecution_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_UpdateWorkflowExecution_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/UpdateWorkflowExecution", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/workflows/{workflow_execution.workflow_id}/update/{request.input.name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_UpdateWorkflowExecution_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_UpdateWorkflowExecution_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_StartBatchOperation_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/StartBatchOperation", runtime.WithHTTPPathPattern("/namespaces/{namespace}/batch-operations/{job_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_StartBatchOperation_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_StartBatchOperation_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_StartBatchOperation_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/StartBatchOperation", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/batch-operations/{job_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_StartBatchOperation_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_StartBatchOperation_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_StopBatchOperation_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/StopBatchOperation", runtime.WithHTTPPathPattern("/namespaces/{namespace}/batch-operations/{job_id}/stop")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_StopBatchOperation_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_StopBatchOperation_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("POST", pattern_WorkflowService_StopBatchOperation_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/StopBatchOperation", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/batch-operations/{job_id}/stop")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_StopBatchOperation_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_StopBatchOperation_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeBatchOperation_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeBatchOperation", runtime.WithHTTPPathPattern("/namespaces/{namespace}/batch-operations/{job_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_DescribeBatchOperation_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeBatchOperation_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_DescribeBatchOperation_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/DescribeBatchOperation", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/batch-operations/{job_id}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_DescribeBatchOperation_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_DescribeBatchOperation_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListBatchOperations_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListBatchOperations", runtime.WithHTTPPathPattern("/namespaces/{namespace}/batch-operations")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ListBatchOperations_0(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListBatchOperations_0(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_WorkflowService_ListBatchOperations_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + var err error + var annotatedContext context.Context + annotatedContext, err = runtime.AnnotateContext(ctx, mux, req, "/temporal.api.workflowservice.v1.WorkflowService/ListBatchOperations", runtime.WithHTTPPathPattern("/api/v1/namespaces/{namespace}/batch-operations")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_WorkflowService_ListBatchOperations_1(annotatedContext, inboundMarshaler, client, req, pathParams) + annotatedContext = runtime.NewServerMetadataContext(annotatedContext, md) + if err != nil { + runtime.HTTPError(annotatedContext, mux, outboundMarshaler, w, req, err) + return + } + + forward_WorkflowService_ListBatchOperations_1(annotatedContext, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + return nil +} + +var ( + pattern_WorkflowService_RegisterNamespace_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"cluster", "namespaces"}, "")) + + pattern_WorkflowService_RegisterNamespace_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"api", "v1", "namespaces"}, "")) + + pattern_WorkflowService_DescribeNamespace_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2}, []string{"cluster", "namespaces", "namespace"}, "")) + + pattern_WorkflowService_DescribeNamespace_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"api", "v1", "namespaces", "namespace"}, "")) + + pattern_WorkflowService_ListNamespaces_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"cluster", "namespaces"}, "")) + + pattern_WorkflowService_ListNamespaces_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"api", "v1", "namespaces"}, "")) + + pattern_WorkflowService_UpdateNamespace_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 1, 0, 4, 1, 5, 2, 2, 3}, []string{"cluster", "namespaces", "namespace", "update"}, "")) + + pattern_WorkflowService_UpdateNamespace_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"api", "v1", "namespaces", "namespace", "update"}, "")) + + pattern_WorkflowService_StartWorkflowExecution_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"namespaces", "namespace", "workflows", "workflow_id"}, "")) + + pattern_WorkflowService_StartWorkflowExecution_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"api", "v1", "namespaces", "namespace", "workflows", "workflow_id"}, "")) + + pattern_WorkflowService_ExecuteMultiOperation_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 2, 3}, []string{"namespaces", "namespace", "workflows", "execute-multi-operation"}, "")) + + pattern_WorkflowService_ExecuteMultiOperation_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 2, 5}, []string{"api", "v1", "namespaces", "namespace", "workflows", "execute-multi-operation"}, "")) + + pattern_WorkflowService_GetWorkflowExecutionHistory_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"namespaces", "namespace", "workflows", "execution.workflow_id", "history"}, "")) + + pattern_WorkflowService_GetWorkflowExecutionHistory_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6}, []string{"api", "v1", "namespaces", "namespace", "workflows", "execution.workflow_id", "history"}, "")) + + pattern_WorkflowService_GetWorkflowExecutionHistoryReverse_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"namespaces", "namespace", "workflows", "execution.workflow_id", "history-reverse"}, "")) + + pattern_WorkflowService_GetWorkflowExecutionHistoryReverse_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6}, []string{"api", "v1", "namespaces", "namespace", "workflows", "execution.workflow_id", "history-reverse"}, "")) + + pattern_WorkflowService_RecordActivityTaskHeartbeat_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 2, 3}, []string{"namespaces", "namespace", "activities", "heartbeat"}, "")) + + pattern_WorkflowService_RecordActivityTaskHeartbeat_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 2, 5}, []string{"api", "v1", "namespaces", "namespace", "activities", "heartbeat"}, "")) + + pattern_WorkflowService_RecordActivityTaskHeartbeatById_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 2, 3}, []string{"namespaces", "namespace", "activities", "heartbeat-by-id"}, "")) + + pattern_WorkflowService_RecordActivityTaskHeartbeatById_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 2, 5}, []string{"api", "v1", "namespaces", "namespace", "activities", "heartbeat-by-id"}, "")) + + pattern_WorkflowService_RespondActivityTaskCompleted_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 2, 3}, []string{"namespaces", "namespace", "activities", "complete"}, "")) + + pattern_WorkflowService_RespondActivityTaskCompleted_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 2, 5}, []string{"api", "v1", "namespaces", "namespace", "activities", "complete"}, "")) + + pattern_WorkflowService_RespondActivityTaskCompletedById_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 2, 3}, []string{"namespaces", "namespace", "activities", "complete-by-id"}, "")) + + pattern_WorkflowService_RespondActivityTaskCompletedById_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 2, 5}, []string{"api", "v1", "namespaces", "namespace", "activities", "complete-by-id"}, "")) + + pattern_WorkflowService_RespondActivityTaskFailed_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 2, 3}, []string{"namespaces", "namespace", "activities", "fail"}, "")) + + pattern_WorkflowService_RespondActivityTaskFailed_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 2, 5}, []string{"api", "v1", "namespaces", "namespace", "activities", "fail"}, "")) + + pattern_WorkflowService_RespondActivityTaskFailedById_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 2, 3}, []string{"namespaces", "namespace", "activities", "fail-by-id"}, "")) + + pattern_WorkflowService_RespondActivityTaskFailedById_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 2, 5}, []string{"api", "v1", "namespaces", "namespace", "activities", "fail-by-id"}, "")) + + pattern_WorkflowService_RespondActivityTaskCanceled_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 2, 3}, []string{"namespaces", "namespace", "activities", "cancel"}, "")) + + pattern_WorkflowService_RespondActivityTaskCanceled_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 2, 5}, []string{"api", "v1", "namespaces", "namespace", "activities", "cancel"}, "")) + + pattern_WorkflowService_RespondActivityTaskCanceledById_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 2, 3}, []string{"namespaces", "namespace", "activities", "cancel-by-id"}, "")) + + pattern_WorkflowService_RespondActivityTaskCanceledById_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 2, 5}, []string{"api", "v1", "namespaces", "namespace", "activities", "cancel-by-id"}, "")) + + pattern_WorkflowService_RequestCancelWorkflowExecution_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"namespaces", "namespace", "workflows", "workflow_execution.workflow_id", "cancel"}, "")) + + pattern_WorkflowService_RequestCancelWorkflowExecution_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6}, []string{"api", "v1", "namespaces", "namespace", "workflows", "workflow_execution.workflow_id", "cancel"}, "")) + + pattern_WorkflowService_SignalWorkflowExecution_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"namespaces", "namespace", "workflows", "workflow_execution.workflow_id", "signal", "signal_name"}, "")) + + pattern_WorkflowService_SignalWorkflowExecution_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6, 1, 0, 4, 1, 5, 7}, []string{"api", "v1", "namespaces", "namespace", "workflows", "workflow_execution.workflow_id", "signal", "signal_name"}, "")) + + pattern_WorkflowService_SignalWithStartWorkflowExecution_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"namespaces", "namespace", "workflows", "workflow_id", "signal-with-start", "signal_name"}, "")) + + pattern_WorkflowService_SignalWithStartWorkflowExecution_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6, 1, 0, 4, 1, 5, 7}, []string{"api", "v1", "namespaces", "namespace", "workflows", "workflow_id", "signal-with-start", "signal_name"}, "")) + + pattern_WorkflowService_ResetWorkflowExecution_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"namespaces", "namespace", "workflows", "workflow_execution.workflow_id", "reset"}, "")) + + pattern_WorkflowService_ResetWorkflowExecution_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6}, []string{"api", "v1", "namespaces", "namespace", "workflows", "workflow_execution.workflow_id", "reset"}, "")) + + pattern_WorkflowService_TerminateWorkflowExecution_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"namespaces", "namespace", "workflows", "workflow_execution.workflow_id", "terminate"}, "")) + + pattern_WorkflowService_TerminateWorkflowExecution_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6}, []string{"api", "v1", "namespaces", "namespace", "workflows", "workflow_execution.workflow_id", "terminate"}, "")) + + pattern_WorkflowService_ListWorkflowExecutions_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2}, []string{"namespaces", "namespace", "workflows"}, "")) + + pattern_WorkflowService_ListWorkflowExecutions_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"api", "v1", "namespaces", "namespace", "workflows"}, "")) + + pattern_WorkflowService_ListArchivedWorkflowExecutions_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2}, []string{"namespaces", "namespace", "archived-workflows"}, "")) + + pattern_WorkflowService_ListArchivedWorkflowExecutions_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"api", "v1", "namespaces", "namespace", "archived-workflows"}, "")) + + pattern_WorkflowService_CountWorkflowExecutions_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2}, []string{"namespaces", "namespace", "workflow-count"}, "")) + + pattern_WorkflowService_CountWorkflowExecutions_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"api", "v1", "namespaces", "namespace", "workflow-count"}, "")) + + pattern_WorkflowService_QueryWorkflow_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"namespaces", "namespace", "workflows", "execution.workflow_id", "query", "query.query_type"}, "")) + + pattern_WorkflowService_QueryWorkflow_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6, 1, 0, 4, 1, 5, 7}, []string{"api", "v1", "namespaces", "namespace", "workflows", "execution.workflow_id", "query", "query.query_type"}, "")) + + pattern_WorkflowService_DescribeWorkflowExecution_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"namespaces", "namespace", "workflows", "execution.workflow_id"}, "")) + + pattern_WorkflowService_DescribeWorkflowExecution_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"api", "v1", "namespaces", "namespace", "workflows", "execution.workflow_id"}, "")) + + pattern_WorkflowService_DescribeTaskQueue_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"namespaces", "namespace", "task-queues", "task_queue.name"}, "")) + + pattern_WorkflowService_DescribeTaskQueue_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"api", "v1", "namespaces", "namespace", "task-queues", "task_queue.name"}, "")) + + pattern_WorkflowService_GetClusterInfo_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0}, []string{"cluster"}, "")) + + pattern_WorkflowService_GetClusterInfo_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"api", "v1", "cluster-info"}, "")) + + pattern_WorkflowService_GetSystemInfo_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0}, []string{"system-info"}, "")) + + pattern_WorkflowService_GetSystemInfo_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"api", "v1", "system-info"}, "")) + + pattern_WorkflowService_CreateSchedule_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"namespaces", "namespace", "schedules", "schedule_id"}, "")) + + pattern_WorkflowService_CreateSchedule_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"api", "v1", "namespaces", "namespace", "schedules", "schedule_id"}, "")) + + pattern_WorkflowService_DescribeSchedule_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"namespaces", "namespace", "schedules", "schedule_id"}, "")) + + pattern_WorkflowService_DescribeSchedule_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"api", "v1", "namespaces", "namespace", "schedules", "schedule_id"}, "")) + + pattern_WorkflowService_UpdateSchedule_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"namespaces", "namespace", "schedules", "schedule_id", "update"}, "")) + + pattern_WorkflowService_UpdateSchedule_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6}, []string{"api", "v1", "namespaces", "namespace", "schedules", "schedule_id", "update"}, "")) + + pattern_WorkflowService_PatchSchedule_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"namespaces", "namespace", "schedules", "schedule_id", "patch"}, "")) + + pattern_WorkflowService_PatchSchedule_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6}, []string{"api", "v1", "namespaces", "namespace", "schedules", "schedule_id", "patch"}, "")) + + pattern_WorkflowService_ListScheduleMatchingTimes_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"namespaces", "namespace", "schedules", "schedule_id", "matching-times"}, "")) + + pattern_WorkflowService_ListScheduleMatchingTimes_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6}, []string{"api", "v1", "namespaces", "namespace", "schedules", "schedule_id", "matching-times"}, "")) + + pattern_WorkflowService_DeleteSchedule_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"namespaces", "namespace", "schedules", "schedule_id"}, "")) + + pattern_WorkflowService_DeleteSchedule_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"api", "v1", "namespaces", "namespace", "schedules", "schedule_id"}, "")) + + pattern_WorkflowService_ListSchedules_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2}, []string{"namespaces", "namespace", "schedules"}, "")) + + pattern_WorkflowService_ListSchedules_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"api", "v1", "namespaces", "namespace", "schedules"}, "")) + + pattern_WorkflowService_GetWorkerBuildIdCompatibility_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"namespaces", "namespace", "task-queues", "task_queue", "worker-build-id-compatibility"}, "")) + + pattern_WorkflowService_GetWorkerBuildIdCompatibility_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6}, []string{"api", "v1", "namespaces", "namespace", "task-queues", "task_queue", "worker-build-id-compatibility"}, "")) + + pattern_WorkflowService_GetWorkerVersioningRules_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"namespaces", "namespace", "task-queues", "task_queue", "worker-versioning-rules"}, "")) + + pattern_WorkflowService_GetWorkerVersioningRules_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6}, []string{"api", "v1", "namespaces", "namespace", "task-queues", "task_queue", "worker-versioning-rules"}, "")) + + pattern_WorkflowService_GetWorkerTaskReachability_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2}, []string{"namespaces", "namespace", "worker-task-reachability"}, "")) + + pattern_WorkflowService_GetWorkerTaskReachability_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"api", "v1", "namespaces", "namespace", "worker-task-reachability"}, "")) + + pattern_WorkflowService_UpdateWorkflowExecution_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"namespaces", "namespace", "workflows", "workflow_execution.workflow_id", "update", "request.input.name"}, "")) + + pattern_WorkflowService_UpdateWorkflowExecution_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6, 1, 0, 4, 1, 5, 7}, []string{"api", "v1", "namespaces", "namespace", "workflows", "workflow_execution.workflow_id", "update", "request.input.name"}, "")) + + pattern_WorkflowService_StartBatchOperation_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"namespaces", "namespace", "batch-operations", "job_id"}, "")) + + pattern_WorkflowService_StartBatchOperation_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"api", "v1", "namespaces", "namespace", "batch-operations", "job_id"}, "")) + + pattern_WorkflowService_StopBatchOperation_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"namespaces", "namespace", "batch-operations", "job_id", "stop"}, "")) + + pattern_WorkflowService_StopBatchOperation_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5, 2, 6}, []string{"api", "v1", "namespaces", "namespace", "batch-operations", "job_id", "stop"}, "")) + + pattern_WorkflowService_DescribeBatchOperation_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"namespaces", "namespace", "batch-operations", "job_id"}, "")) + + pattern_WorkflowService_DescribeBatchOperation_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"api", "v1", "namespaces", "namespace", "batch-operations", "job_id"}, "")) + + pattern_WorkflowService_ListBatchOperations_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 1, 0, 4, 1, 5, 1, 2, 2}, []string{"namespaces", "namespace", "batch-operations"}, "")) + + pattern_WorkflowService_ListBatchOperations_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"api", "v1", "namespaces", "namespace", "batch-operations"}, "")) +) + +var ( + forward_WorkflowService_RegisterNamespace_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_RegisterNamespace_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_DescribeNamespace_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_DescribeNamespace_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_ListNamespaces_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_ListNamespaces_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_UpdateNamespace_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_UpdateNamespace_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_StartWorkflowExecution_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_StartWorkflowExecution_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_ExecuteMultiOperation_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_ExecuteMultiOperation_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetWorkflowExecutionHistory_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetWorkflowExecutionHistory_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetWorkflowExecutionHistoryReverse_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetWorkflowExecutionHistoryReverse_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_RecordActivityTaskHeartbeat_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_RecordActivityTaskHeartbeat_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_RecordActivityTaskHeartbeatById_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_RecordActivityTaskHeartbeatById_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_RespondActivityTaskCompleted_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_RespondActivityTaskCompleted_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_RespondActivityTaskCompletedById_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_RespondActivityTaskCompletedById_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_RespondActivityTaskFailed_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_RespondActivityTaskFailed_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_RespondActivityTaskFailedById_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_RespondActivityTaskFailedById_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_RespondActivityTaskCanceled_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_RespondActivityTaskCanceled_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_RespondActivityTaskCanceledById_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_RespondActivityTaskCanceledById_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_RequestCancelWorkflowExecution_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_RequestCancelWorkflowExecution_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_SignalWorkflowExecution_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_SignalWorkflowExecution_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_SignalWithStartWorkflowExecution_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_SignalWithStartWorkflowExecution_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_ResetWorkflowExecution_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_ResetWorkflowExecution_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_TerminateWorkflowExecution_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_TerminateWorkflowExecution_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_ListWorkflowExecutions_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_ListWorkflowExecutions_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_ListArchivedWorkflowExecutions_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_ListArchivedWorkflowExecutions_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_CountWorkflowExecutions_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_CountWorkflowExecutions_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_QueryWorkflow_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_QueryWorkflow_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_DescribeWorkflowExecution_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_DescribeWorkflowExecution_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_DescribeTaskQueue_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_DescribeTaskQueue_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetClusterInfo_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetClusterInfo_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetSystemInfo_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetSystemInfo_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_CreateSchedule_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_CreateSchedule_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_DescribeSchedule_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_DescribeSchedule_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_UpdateSchedule_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_UpdateSchedule_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_PatchSchedule_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_PatchSchedule_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_ListScheduleMatchingTimes_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_ListScheduleMatchingTimes_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_DeleteSchedule_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_DeleteSchedule_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_ListSchedules_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_ListSchedules_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetWorkerBuildIdCompatibility_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetWorkerBuildIdCompatibility_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetWorkerVersioningRules_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetWorkerVersioningRules_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetWorkerTaskReachability_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_GetWorkerTaskReachability_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_UpdateWorkflowExecution_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_UpdateWorkflowExecution_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_StartBatchOperation_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_StartBatchOperation_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_StopBatchOperation_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_StopBatchOperation_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_DescribeBatchOperation_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_DescribeBatchOperation_1 = runtime.ForwardResponseMessage + + forward_WorkflowService_ListBatchOperations_0 = runtime.ForwardResponseMessage + + forward_WorkflowService_ListBatchOperations_1 = runtime.ForwardResponseMessage +) diff --git a/vendor/go.temporal.io/api/workflowservice/v1/service_grpc.pb.go b/vendor/go.temporal.io/api/workflowservice/v1/service_grpc.pb.go new file mode 100644 index 00000000000..cec1c02bf69 --- /dev/null +++ b/vendor/go.temporal.io/api/workflowservice/v1/service_grpc.pb.go @@ -0,0 +1,3160 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// plugins: +// - protoc-gen-go-grpc +// - protoc +// source: temporal/api/workflowservice/v1/service.proto + +package workflowservice + +import ( + context "context" + + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.64.0 or later. +const _ = grpc.SupportPackageIsVersion9 + +const ( + WorkflowService_RegisterNamespace_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RegisterNamespace" + WorkflowService_DescribeNamespace_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/DescribeNamespace" + WorkflowService_ListNamespaces_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/ListNamespaces" + WorkflowService_UpdateNamespace_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/UpdateNamespace" + WorkflowService_DeprecateNamespace_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/DeprecateNamespace" + WorkflowService_StartWorkflowExecution_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/StartWorkflowExecution" + WorkflowService_ExecuteMultiOperation_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/ExecuteMultiOperation" + WorkflowService_GetWorkflowExecutionHistory_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/GetWorkflowExecutionHistory" + WorkflowService_GetWorkflowExecutionHistoryReverse_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/GetWorkflowExecutionHistoryReverse" + WorkflowService_PollWorkflowTaskQueue_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/PollWorkflowTaskQueue" + WorkflowService_RespondWorkflowTaskCompleted_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RespondWorkflowTaskCompleted" + WorkflowService_RespondWorkflowTaskFailed_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RespondWorkflowTaskFailed" + WorkflowService_PollActivityTaskQueue_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/PollActivityTaskQueue" + WorkflowService_RecordActivityTaskHeartbeat_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RecordActivityTaskHeartbeat" + WorkflowService_RecordActivityTaskHeartbeatById_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RecordActivityTaskHeartbeatById" + WorkflowService_RespondActivityTaskCompleted_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCompleted" + WorkflowService_RespondActivityTaskCompletedById_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCompletedById" + WorkflowService_RespondActivityTaskFailed_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskFailed" + WorkflowService_RespondActivityTaskFailedById_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskFailedById" + WorkflowService_RespondActivityTaskCanceled_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCanceled" + WorkflowService_RespondActivityTaskCanceledById_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RespondActivityTaskCanceledById" + WorkflowService_RequestCancelWorkflowExecution_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RequestCancelWorkflowExecution" + WorkflowService_SignalWorkflowExecution_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/SignalWorkflowExecution" + WorkflowService_SignalWithStartWorkflowExecution_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/SignalWithStartWorkflowExecution" + WorkflowService_ResetWorkflowExecution_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/ResetWorkflowExecution" + WorkflowService_TerminateWorkflowExecution_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/TerminateWorkflowExecution" + WorkflowService_DeleteWorkflowExecution_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/DeleteWorkflowExecution" + WorkflowService_ListOpenWorkflowExecutions_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/ListOpenWorkflowExecutions" + WorkflowService_ListClosedWorkflowExecutions_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/ListClosedWorkflowExecutions" + WorkflowService_ListWorkflowExecutions_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/ListWorkflowExecutions" + WorkflowService_ListArchivedWorkflowExecutions_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/ListArchivedWorkflowExecutions" + WorkflowService_ScanWorkflowExecutions_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/ScanWorkflowExecutions" + WorkflowService_CountWorkflowExecutions_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/CountWorkflowExecutions" + WorkflowService_GetSearchAttributes_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/GetSearchAttributes" + WorkflowService_RespondQueryTaskCompleted_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RespondQueryTaskCompleted" + WorkflowService_ResetStickyTaskQueue_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/ResetStickyTaskQueue" + WorkflowService_QueryWorkflow_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/QueryWorkflow" + WorkflowService_DescribeWorkflowExecution_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/DescribeWorkflowExecution" + WorkflowService_DescribeTaskQueue_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/DescribeTaskQueue" + WorkflowService_GetClusterInfo_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/GetClusterInfo" + WorkflowService_GetSystemInfo_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/GetSystemInfo" + WorkflowService_ListTaskQueuePartitions_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/ListTaskQueuePartitions" + WorkflowService_CreateSchedule_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/CreateSchedule" + WorkflowService_DescribeSchedule_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/DescribeSchedule" + WorkflowService_UpdateSchedule_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/UpdateSchedule" + WorkflowService_PatchSchedule_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/PatchSchedule" + WorkflowService_ListScheduleMatchingTimes_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/ListScheduleMatchingTimes" + WorkflowService_DeleteSchedule_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/DeleteSchedule" + WorkflowService_ListSchedules_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/ListSchedules" + WorkflowService_UpdateWorkerBuildIdCompatibility_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/UpdateWorkerBuildIdCompatibility" + WorkflowService_GetWorkerBuildIdCompatibility_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerBuildIdCompatibility" + WorkflowService_UpdateWorkerVersioningRules_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/UpdateWorkerVersioningRules" + WorkflowService_GetWorkerVersioningRules_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerVersioningRules" + WorkflowService_GetWorkerTaskReachability_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/GetWorkerTaskReachability" + WorkflowService_UpdateWorkflowExecution_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/UpdateWorkflowExecution" + WorkflowService_PollWorkflowExecutionUpdate_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/PollWorkflowExecutionUpdate" + WorkflowService_StartBatchOperation_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/StartBatchOperation" + WorkflowService_StopBatchOperation_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/StopBatchOperation" + WorkflowService_DescribeBatchOperation_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/DescribeBatchOperation" + WorkflowService_ListBatchOperations_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/ListBatchOperations" + WorkflowService_PollNexusTaskQueue_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/PollNexusTaskQueue" + WorkflowService_RespondNexusTaskCompleted_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RespondNexusTaskCompleted" + WorkflowService_RespondNexusTaskFailed_FullMethodName = "/temporal.api.workflowservice.v1.WorkflowService/RespondNexusTaskFailed" +) + +// WorkflowServiceClient is the client API for WorkflowService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// WorkflowService API defines how Temporal SDKs and other clients interact with the Temporal server +// to create and interact with workflows and activities. +// +// Users are expected to call `StartWorkflowExecution` to create a new workflow execution. +// +// To drive workflows, a worker using a Temporal SDK must exist which regularly polls for workflow +// and activity tasks from the service. For each workflow task, the sdk must process the +// (incremental or complete) event history and respond back with any newly generated commands. +// +// For each activity task, the worker is expected to execute the user's code which implements that +// activity, responding with completion or failure. +type WorkflowServiceClient interface { + // RegisterNamespace creates a new namespace which can be used as a container for all resources. + // + // A Namespace is a top level entity within Temporal, and is used as a container for resources + // like workflow executions, task queues, etc. A Namespace acts as a sandbox and provides + // isolation for all resources within the namespace. All resources belongs to exactly one + // namespace. + RegisterNamespace(ctx context.Context, in *RegisterNamespaceRequest, opts ...grpc.CallOption) (*RegisterNamespaceResponse, error) + // DescribeNamespace returns the information and configuration for a registered namespace. + DescribeNamespace(ctx context.Context, in *DescribeNamespaceRequest, opts ...grpc.CallOption) (*DescribeNamespaceResponse, error) + // ListNamespaces returns the information and configuration for all namespaces. + ListNamespaces(ctx context.Context, in *ListNamespacesRequest, opts ...grpc.CallOption) (*ListNamespacesResponse, error) + // UpdateNamespace is used to update the information and configuration of a registered + // namespace. + UpdateNamespace(ctx context.Context, in *UpdateNamespaceRequest, opts ...grpc.CallOption) (*UpdateNamespaceResponse, error) + // DeprecateNamespace is used to update the state of a registered namespace to DEPRECATED. + // + // Once the namespace is deprecated it cannot be used to start new workflow executions. Existing + // workflow executions will continue to run on deprecated namespaces. + // Deprecated. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: Deprecated --) + DeprecateNamespace(ctx context.Context, in *DeprecateNamespaceRequest, opts ...grpc.CallOption) (*DeprecateNamespaceResponse, error) + // StartWorkflowExecution starts a new workflow execution. + // + // It will create the execution with a `WORKFLOW_EXECUTION_STARTED` event in its history and + // also schedule the first workflow task. Returns `WorkflowExecutionAlreadyStarted`, if an + // instance already exists with same workflow id. + StartWorkflowExecution(ctx context.Context, in *StartWorkflowExecutionRequest, opts ...grpc.CallOption) (*StartWorkflowExecutionResponse, error) + // ExecuteMultiOperation executes multiple operations within a single workflow. + // + // Operations are started atomically, meaning if *any* operation fails to be started, none are, + // and the request fails. Upon start, the API returns only when *all* operations have a response. + // + // Upon failure, it returns `MultiOperationExecutionFailure` where the status code + // equals the status code of the *first* operation that failed to be started. + // + // NOTE: Experimental API. + ExecuteMultiOperation(ctx context.Context, in *ExecuteMultiOperationRequest, opts ...grpc.CallOption) (*ExecuteMultiOperationResponse, error) + // GetWorkflowExecutionHistory returns the history of specified workflow execution. Fails with + // `NotFound` if the specified workflow execution is unknown to the service. + GetWorkflowExecutionHistory(ctx context.Context, in *GetWorkflowExecutionHistoryRequest, opts ...grpc.CallOption) (*GetWorkflowExecutionHistoryResponse, error) + // GetWorkflowExecutionHistoryReverse returns the history of specified workflow execution in reverse + // order (starting from last event). Fails with`NotFound` if the specified workflow execution is + // unknown to the service. + GetWorkflowExecutionHistoryReverse(ctx context.Context, in *GetWorkflowExecutionHistoryReverseRequest, opts ...grpc.CallOption) (*GetWorkflowExecutionHistoryReverseResponse, error) + // PollWorkflowTaskQueue is called by workers to make progress on workflows. + // + // A WorkflowTask is dispatched to callers for active workflow executions with pending workflow + // tasks. The worker is expected to call `RespondWorkflowTaskCompleted` when it is done + // processing the task. The service will create a `WorkflowTaskStarted` event in the history for + // this task before handing it to the worker. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + PollWorkflowTaskQueue(ctx context.Context, in *PollWorkflowTaskQueueRequest, opts ...grpc.CallOption) (*PollWorkflowTaskQueueResponse, error) + // RespondWorkflowTaskCompleted is called by workers to successfully complete workflow tasks + // they received from `PollWorkflowTaskQueue`. + // + // Completing a WorkflowTask will write a `WORKFLOW_TASK_COMPLETED` event to the workflow's + // history, along with events corresponding to whatever commands the SDK generated while + // executing the task (ex timer started, activity task scheduled, etc). + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + RespondWorkflowTaskCompleted(ctx context.Context, in *RespondWorkflowTaskCompletedRequest, opts ...grpc.CallOption) (*RespondWorkflowTaskCompletedResponse, error) + // RespondWorkflowTaskFailed is called by workers to indicate the processing of a workflow task + // failed. + // + // This results in a `WORKFLOW_TASK_FAILED` event written to the history, and a new workflow + // task will be scheduled. This API can be used to report unhandled failures resulting from + // applying the workflow task. + // + // Temporal will only append first WorkflowTaskFailed event to the history of workflow execution + // for consecutive failures. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + RespondWorkflowTaskFailed(ctx context.Context, in *RespondWorkflowTaskFailedRequest, opts ...grpc.CallOption) (*RespondWorkflowTaskFailedResponse, error) + // PollActivityTaskQueue is called by workers to process activity tasks from a specific task + // queue. + // + // The worker is expected to call one of the `RespondActivityTaskXXX` methods when it is done + // processing the task. + // + // An activity task is dispatched whenever a `SCHEDULE_ACTIVITY_TASK` command is produced during + // workflow execution. An in memory `ACTIVITY_TASK_STARTED` event is written to mutable state + // before the task is dispatched to the worker. The started event, and the final event + // (`ACTIVITY_TASK_COMPLETED` / `ACTIVITY_TASK_FAILED` / `ACTIVITY_TASK_TIMED_OUT`) will both be + // written permanently to Workflow execution history when Activity is finished. This is done to + // avoid writing many events in the case of a failure/retry loop. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + PollActivityTaskQueue(ctx context.Context, in *PollActivityTaskQueueRequest, opts ...grpc.CallOption) (*PollActivityTaskQueueResponse, error) + // RecordActivityTaskHeartbeat is optionally called by workers while they execute activities. + // + // If worker fails to heartbeat within the `heartbeat_timeout` interval for the activity task, + // then it will be marked as timed out and an `ACTIVITY_TASK_TIMED_OUT` event will be written to + // the workflow history. Calling `RecordActivityTaskHeartbeat` will fail with `NotFound` in + // such situations, in that event, the SDK should request cancellation of the activity. + RecordActivityTaskHeartbeat(ctx context.Context, in *RecordActivityTaskHeartbeatRequest, opts ...grpc.CallOption) (*RecordActivityTaskHeartbeatResponse, error) + // See `RecordActivityTaskHeartbeat`. This version allows clients to record heartbeats by + // namespace/workflow id/activity id instead of task token. + // + // (-- api-linter: core::0136::prepositions=disabled + // + // aip.dev/not-precedent: "By" is used to indicate request type. --) + RecordActivityTaskHeartbeatById(ctx context.Context, in *RecordActivityTaskHeartbeatByIdRequest, opts ...grpc.CallOption) (*RecordActivityTaskHeartbeatByIdResponse, error) + // RespondActivityTaskCompleted is called by workers when they successfully complete an activity + // task. + // + // This results in a new `ACTIVITY_TASK_COMPLETED` event being written to the workflow history + // and a new workflow task created for the workflow. Fails with `NotFound` if the task token is + // no longer valid due to activity timeout, already being completed, or never having existed. + RespondActivityTaskCompleted(ctx context.Context, in *RespondActivityTaskCompletedRequest, opts ...grpc.CallOption) (*RespondActivityTaskCompletedResponse, error) + // See `RecordActivityTaskCompleted`. This version allows clients to record completions by + // namespace/workflow id/activity id instead of task token. + // + // (-- api-linter: core::0136::prepositions=disabled + // + // aip.dev/not-precedent: "By" is used to indicate request type. --) + RespondActivityTaskCompletedById(ctx context.Context, in *RespondActivityTaskCompletedByIdRequest, opts ...grpc.CallOption) (*RespondActivityTaskCompletedByIdResponse, error) + // RespondActivityTaskFailed is called by workers when processing an activity task fails. + // + // This results in a new `ACTIVITY_TASK_FAILED` event being written to the workflow history and + // a new workflow task created for the workflow. Fails with `NotFound` if the task token is no + // longer valid due to activity timeout, already being completed, or never having existed. + RespondActivityTaskFailed(ctx context.Context, in *RespondActivityTaskFailedRequest, opts ...grpc.CallOption) (*RespondActivityTaskFailedResponse, error) + // See `RecordActivityTaskFailed`. This version allows clients to record failures by + // namespace/workflow id/activity id instead of task token. + // + // (-- api-linter: core::0136::prepositions=disabled + // + // aip.dev/not-precedent: "By" is used to indicate request type. --) + RespondActivityTaskFailedById(ctx context.Context, in *RespondActivityTaskFailedByIdRequest, opts ...grpc.CallOption) (*RespondActivityTaskFailedByIdResponse, error) + // RespondActivityTaskFailed is called by workers when processing an activity task fails. + // + // This results in a new `ACTIVITY_TASK_CANCELED` event being written to the workflow history + // and a new workflow task created for the workflow. Fails with `NotFound` if the task token is + // no longer valid due to activity timeout, already being completed, or never having existed. + RespondActivityTaskCanceled(ctx context.Context, in *RespondActivityTaskCanceledRequest, opts ...grpc.CallOption) (*RespondActivityTaskCanceledResponse, error) + // See `RecordActivityTaskCanceled`. This version allows clients to record failures by + // namespace/workflow id/activity id instead of task token. + // + // (-- api-linter: core::0136::prepositions=disabled + // + // aip.dev/not-precedent: "By" is used to indicate request type. --) + RespondActivityTaskCanceledById(ctx context.Context, in *RespondActivityTaskCanceledByIdRequest, opts ...grpc.CallOption) (*RespondActivityTaskCanceledByIdResponse, error) + // RequestCancelWorkflowExecution is called by workers when they want to request cancellation of + // a workflow execution. + // + // This results in a new `WORKFLOW_EXECUTION_CANCEL_REQUESTED` event being written to the + // workflow history and a new workflow task created for the workflow. It returns success if the requested + // workflow is already closed. It fails with 'NotFound' if the requested workflow doesn't exist. + RequestCancelWorkflowExecution(ctx context.Context, in *RequestCancelWorkflowExecutionRequest, opts ...grpc.CallOption) (*RequestCancelWorkflowExecutionResponse, error) + // SignalWorkflowExecution is used to send a signal to a running workflow execution. + // + // This results in a `WORKFLOW_EXECUTION_SIGNALED` event recorded in the history and a workflow + // task being created for the execution. + SignalWorkflowExecution(ctx context.Context, in *SignalWorkflowExecutionRequest, opts ...grpc.CallOption) (*SignalWorkflowExecutionResponse, error) + // SignalWithStartWorkflowExecution is used to ensure a signal is sent to a workflow, even if + // it isn't yet started. + // + // If the workflow is running, a `WORKFLOW_EXECUTION_SIGNALED` event is recorded in the history + // and a workflow task is generated. + // + // If the workflow is not running or not found, then the workflow is created with + // `WORKFLOW_EXECUTION_STARTED` and `WORKFLOW_EXECUTION_SIGNALED` events in its history, and a + // workflow task is generated. + // + // (-- api-linter: core::0136::prepositions=disabled + // + // aip.dev/not-precedent: "With" is used to indicate combined operation. --) + SignalWithStartWorkflowExecution(ctx context.Context, in *SignalWithStartWorkflowExecutionRequest, opts ...grpc.CallOption) (*SignalWithStartWorkflowExecutionResponse, error) + // ResetWorkflowExecution will reset an existing workflow execution to a specified + // `WORKFLOW_TASK_COMPLETED` event (exclusive). It will immediately terminate the current + // execution instance. + // TODO: Does exclusive here mean *just* the completed event, or also WFT started? Otherwise the task is doomed to time out? + ResetWorkflowExecution(ctx context.Context, in *ResetWorkflowExecutionRequest, opts ...grpc.CallOption) (*ResetWorkflowExecutionResponse, error) + // TerminateWorkflowExecution terminates an existing workflow execution by recording a + // `WORKFLOW_EXECUTION_TERMINATED` event in the history and immediately terminating the + // execution instance. + TerminateWorkflowExecution(ctx context.Context, in *TerminateWorkflowExecutionRequest, opts ...grpc.CallOption) (*TerminateWorkflowExecutionResponse, error) + // DeleteWorkflowExecution asynchronously deletes a specific Workflow Execution (when + // WorkflowExecution.run_id is provided) or the latest Workflow Execution (when + // WorkflowExecution.run_id is not provided). If the Workflow Execution is Running, it will be + // terminated before deletion. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: Workflow deletion not exposed to HTTP, users should use cancel or terminate. --) + DeleteWorkflowExecution(ctx context.Context, in *DeleteWorkflowExecutionRequest, opts ...grpc.CallOption) (*DeleteWorkflowExecutionResponse, error) + // ListOpenWorkflowExecutions is a visibility API to list the open executions in a specific namespace. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: HTTP users should use ListWorkflowExecutions instead. --) + ListOpenWorkflowExecutions(ctx context.Context, in *ListOpenWorkflowExecutionsRequest, opts ...grpc.CallOption) (*ListOpenWorkflowExecutionsResponse, error) + // ListClosedWorkflowExecutions is a visibility API to list the closed executions in a specific namespace. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: HTTP users should use ListWorkflowExecutions instead. --) + ListClosedWorkflowExecutions(ctx context.Context, in *ListClosedWorkflowExecutionsRequest, opts ...grpc.CallOption) (*ListClosedWorkflowExecutionsResponse, error) + // ListWorkflowExecutions is a visibility API to list workflow executions in a specific namespace. + ListWorkflowExecutions(ctx context.Context, in *ListWorkflowExecutionsRequest, opts ...grpc.CallOption) (*ListWorkflowExecutionsResponse, error) + // ListArchivedWorkflowExecutions is a visibility API to list archived workflow executions in a specific namespace. + ListArchivedWorkflowExecutions(ctx context.Context, in *ListArchivedWorkflowExecutionsRequest, opts ...grpc.CallOption) (*ListArchivedWorkflowExecutionsResponse, error) + // ScanWorkflowExecutions is a visibility API to list large amount of workflow executions in a specific namespace without order. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: HTTP users should use ListWorkflowExecutions instead. --) + ScanWorkflowExecutions(ctx context.Context, in *ScanWorkflowExecutionsRequest, opts ...grpc.CallOption) (*ScanWorkflowExecutionsResponse, error) + // CountWorkflowExecutions is a visibility API to count of workflow executions in a specific namespace. + CountWorkflowExecutions(ctx context.Context, in *CountWorkflowExecutionsRequest, opts ...grpc.CallOption) (*CountWorkflowExecutionsResponse, error) + // GetSearchAttributes is a visibility API to get all legal keys that could be used in list APIs + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose this search attribute API to HTTP (but may expose on OperatorService). --) + GetSearchAttributes(ctx context.Context, in *GetSearchAttributesRequest, opts ...grpc.CallOption) (*GetSearchAttributesResponse, error) + // RespondQueryTaskCompleted is called by workers to complete queries which were delivered on + // the `query` (not `queries`) field of a `PollWorkflowTaskQueueResponse`. + // + // Completing the query will unblock the corresponding client call to `QueryWorkflow` and return + // the query result a response. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + RespondQueryTaskCompleted(ctx context.Context, in *RespondQueryTaskCompletedRequest, opts ...grpc.CallOption) (*RespondQueryTaskCompletedResponse, error) + // ResetStickyTaskQueue resets the sticky task queue related information in the mutable state of + // a given workflow. This is prudent for workers to perform if a workflow has been paged out of + // their cache. + // + // Things cleared are: + // 1. StickyTaskQueue + // 2. StickyScheduleToStartTimeout + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + ResetStickyTaskQueue(ctx context.Context, in *ResetStickyTaskQueueRequest, opts ...grpc.CallOption) (*ResetStickyTaskQueueResponse, error) + // QueryWorkflow requests a query be executed for a specified workflow execution. + QueryWorkflow(ctx context.Context, in *QueryWorkflowRequest, opts ...grpc.CallOption) (*QueryWorkflowResponse, error) + // DescribeWorkflowExecution returns information about the specified workflow execution. + DescribeWorkflowExecution(ctx context.Context, in *DescribeWorkflowExecutionRequest, opts ...grpc.CallOption) (*DescribeWorkflowExecutionResponse, error) + // DescribeTaskQueue returns the following information about the target task queue, broken down by Build ID: + // - List of pollers + // - Workflow Reachability status + // - Backlog info for Workflow and/or Activity tasks + DescribeTaskQueue(ctx context.Context, in *DescribeTaskQueueRequest, opts ...grpc.CallOption) (*DescribeTaskQueueResponse, error) + // GetClusterInfo returns information about temporal cluster + GetClusterInfo(ctx context.Context, in *GetClusterInfoRequest, opts ...grpc.CallOption) (*GetClusterInfoResponse, error) + // GetSystemInfo returns information about the system. + GetSystemInfo(ctx context.Context, in *GetSystemInfoRequest, opts ...grpc.CallOption) (*GetSystemInfoResponse, error) + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose this low-level API to HTTP. --) + ListTaskQueuePartitions(ctx context.Context, in *ListTaskQueuePartitionsRequest, opts ...grpc.CallOption) (*ListTaskQueuePartitionsResponse, error) + // Creates a new schedule. + CreateSchedule(ctx context.Context, in *CreateScheduleRequest, opts ...grpc.CallOption) (*CreateScheduleResponse, error) + // Returns the schedule description and current state of an existing schedule. + DescribeSchedule(ctx context.Context, in *DescribeScheduleRequest, opts ...grpc.CallOption) (*DescribeScheduleResponse, error) + // Changes the configuration or state of an existing schedule. + UpdateSchedule(ctx context.Context, in *UpdateScheduleRequest, opts ...grpc.CallOption) (*UpdateScheduleResponse, error) + // Makes a specific change to a schedule or triggers an immediate action. + PatchSchedule(ctx context.Context, in *PatchScheduleRequest, opts ...grpc.CallOption) (*PatchScheduleResponse, error) + // Lists matching times within a range. + ListScheduleMatchingTimes(ctx context.Context, in *ListScheduleMatchingTimesRequest, opts ...grpc.CallOption) (*ListScheduleMatchingTimesResponse, error) + // Deletes a schedule, removing it from the system. + DeleteSchedule(ctx context.Context, in *DeleteScheduleRequest, opts ...grpc.CallOption) (*DeleteScheduleResponse, error) + // List all schedules in a namespace. + ListSchedules(ctx context.Context, in *ListSchedulesRequest, opts ...grpc.CallOption) (*ListSchedulesResponse, error) + // Deprecated. Use `UpdateWorkerVersioningRules`. + // + // Allows users to specify sets of worker build id versions on a per task queue basis. Versions + // are ordered, and may be either compatible with some extant version, or a new incompatible + // version, forming sets of ids which are incompatible with each other, but whose contained + // members are compatible with one another. + // + // A single build id may be mapped to multiple task queues using this API for cases where a single process hosts + // multiple workers. + // + // To query which workers can be retired, use the `GetWorkerTaskReachability` API. + // + // NOTE: The number of task queues mapped to a single build id is limited by the `limit.taskQueuesPerBuildId` + // (default is 20), if this limit is exceeded this API will error with a FailedPrecondition. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do yet expose versioning API to HTTP. --) + UpdateWorkerBuildIdCompatibility(ctx context.Context, in *UpdateWorkerBuildIdCompatibilityRequest, opts ...grpc.CallOption) (*UpdateWorkerBuildIdCompatibilityResponse, error) + // Deprecated. Use `GetWorkerVersioningRules`. + // Fetches the worker build id versioning sets for a task queue. + GetWorkerBuildIdCompatibility(ctx context.Context, in *GetWorkerBuildIdCompatibilityRequest, opts ...grpc.CallOption) (*GetWorkerBuildIdCompatibilityResponse, error) + // Use this API to manage Worker Versioning Rules for a given Task Queue. There are two types of + // rules: Build ID Assignment rules and Compatible Build ID Redirect rules. + // + // Assignment rules determine how to assign new executions to a Build IDs. Their primary + // use case is to specify the latest Build ID but they have powerful features for gradual rollout + // of a new Build ID. + // + // Once a workflow execution is assigned to a Build ID and it completes its first Workflow Task, + // the workflow stays on the assigned Build ID regardless of changes in assignment rules. This + // eliminates the need for compatibility between versions when you only care about using the new + // version for new workflows and let existing workflows finish in their own version. + // + // Activities, Child Workflows and Continue-as-New executions have the option to inherit the + // Build ID of their parent/previous workflow or use the latest assignment rules to independently + // select a Build ID. + // + // Redirect rules should only be used when you want to move workflows and activities assigned to + // one Build ID (source) to another compatible Build ID (target). You are responsible to make sure + // the target Build ID of a redirect rule is able to process event histories made by the source + // Build ID by using [Patching](https://docs.temporal.io/workflows#patching) or other means. + // + // WARNING: Worker Versioning is not yet stable and the API and behavior may change incompatibly. + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do yet expose versioning API to HTTP. --) + UpdateWorkerVersioningRules(ctx context.Context, in *UpdateWorkerVersioningRulesRequest, opts ...grpc.CallOption) (*UpdateWorkerVersioningRulesResponse, error) + // Fetches the Build ID assignment and redirect rules for a Task Queue. + // WARNING: Worker Versioning is not yet stable and the API and behavior may change incompatibly. + GetWorkerVersioningRules(ctx context.Context, in *GetWorkerVersioningRulesRequest, opts ...grpc.CallOption) (*GetWorkerVersioningRulesResponse, error) + // Deprecated. Use `DescribeTaskQueue`. + // + // Fetches task reachability to determine whether a worker may be retired. + // The request may specify task queues to query for or let the server fetch all task queues mapped to the given + // build IDs. + // + // When requesting a large number of task queues or all task queues associated with the given build ids in a + // namespace, all task queues will be listed in the response but some of them may not contain reachability + // information due to a server enforced limit. When reaching the limit, task queues that reachability information + // could not be retrieved for will be marked with a single TASK_REACHABILITY_UNSPECIFIED entry. The caller may issue + // another call to get the reachability for those task queues. + // + // Open source users can adjust this limit by setting the server's dynamic config value for + // `limit.reachabilityTaskQueueScan` with the caveat that this call can strain the visibility store. + GetWorkerTaskReachability(ctx context.Context, in *GetWorkerTaskReachabilityRequest, opts ...grpc.CallOption) (*GetWorkerTaskReachabilityResponse, error) + // Invokes the specified update function on user workflow code. + UpdateWorkflowExecution(ctx context.Context, in *UpdateWorkflowExecutionRequest, opts ...grpc.CallOption) (*UpdateWorkflowExecutionResponse, error) + // Polls a workflow execution for the outcome of a workflow execution update + // previously issued through the UpdateWorkflowExecution RPC. The effective + // timeout on this call will be shorter of the the caller-supplied gRPC + // timeout and the server's configured long-poll timeout. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We don't expose update polling API to HTTP in favor of a potential future non-blocking form. --) + PollWorkflowExecutionUpdate(ctx context.Context, in *PollWorkflowExecutionUpdateRequest, opts ...grpc.CallOption) (*PollWorkflowExecutionUpdateResponse, error) + // StartBatchOperation starts a new batch operation + StartBatchOperation(ctx context.Context, in *StartBatchOperationRequest, opts ...grpc.CallOption) (*StartBatchOperationResponse, error) + // StopBatchOperation stops a batch operation + StopBatchOperation(ctx context.Context, in *StopBatchOperationRequest, opts ...grpc.CallOption) (*StopBatchOperationResponse, error) + // DescribeBatchOperation returns the information about a batch operation + DescribeBatchOperation(ctx context.Context, in *DescribeBatchOperationRequest, opts ...grpc.CallOption) (*DescribeBatchOperationResponse, error) + // ListBatchOperations returns a list of batch operations + ListBatchOperations(ctx context.Context, in *ListBatchOperationsRequest, opts ...grpc.CallOption) (*ListBatchOperationsResponse, error) + // PollNexusTaskQueue is a long poll call used by workers to receive Nexus tasks. + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + PollNexusTaskQueue(ctx context.Context, in *PollNexusTaskQueueRequest, opts ...grpc.CallOption) (*PollNexusTaskQueueResponse, error) + // RespondNexusTaskCompleted is called by workers to respond to Nexus tasks received via PollNexusTaskQueue. + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + RespondNexusTaskCompleted(ctx context.Context, in *RespondNexusTaskCompletedRequest, opts ...grpc.CallOption) (*RespondNexusTaskCompletedResponse, error) + // RespondNexusTaskFailed is called by workers to fail Nexus tasks received via PollNexusTaskQueue. + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + RespondNexusTaskFailed(ctx context.Context, in *RespondNexusTaskFailedRequest, opts ...grpc.CallOption) (*RespondNexusTaskFailedResponse, error) +} + +type workflowServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewWorkflowServiceClient(cc grpc.ClientConnInterface) WorkflowServiceClient { + return &workflowServiceClient{cc} +} + +func (c *workflowServiceClient) RegisterNamespace(ctx context.Context, in *RegisterNamespaceRequest, opts ...grpc.CallOption) (*RegisterNamespaceResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RegisterNamespaceResponse) + err := c.cc.Invoke(ctx, WorkflowService_RegisterNamespace_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) DescribeNamespace(ctx context.Context, in *DescribeNamespaceRequest, opts ...grpc.CallOption) (*DescribeNamespaceResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DescribeNamespaceResponse) + err := c.cc.Invoke(ctx, WorkflowService_DescribeNamespace_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) ListNamespaces(ctx context.Context, in *ListNamespacesRequest, opts ...grpc.CallOption) (*ListNamespacesResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ListNamespacesResponse) + err := c.cc.Invoke(ctx, WorkflowService_ListNamespaces_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) UpdateNamespace(ctx context.Context, in *UpdateNamespaceRequest, opts ...grpc.CallOption) (*UpdateNamespaceResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(UpdateNamespaceResponse) + err := c.cc.Invoke(ctx, WorkflowService_UpdateNamespace_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) DeprecateNamespace(ctx context.Context, in *DeprecateNamespaceRequest, opts ...grpc.CallOption) (*DeprecateNamespaceResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DeprecateNamespaceResponse) + err := c.cc.Invoke(ctx, WorkflowService_DeprecateNamespace_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) StartWorkflowExecution(ctx context.Context, in *StartWorkflowExecutionRequest, opts ...grpc.CallOption) (*StartWorkflowExecutionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(StartWorkflowExecutionResponse) + err := c.cc.Invoke(ctx, WorkflowService_StartWorkflowExecution_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) ExecuteMultiOperation(ctx context.Context, in *ExecuteMultiOperationRequest, opts ...grpc.CallOption) (*ExecuteMultiOperationResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ExecuteMultiOperationResponse) + err := c.cc.Invoke(ctx, WorkflowService_ExecuteMultiOperation_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) GetWorkflowExecutionHistory(ctx context.Context, in *GetWorkflowExecutionHistoryRequest, opts ...grpc.CallOption) (*GetWorkflowExecutionHistoryResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetWorkflowExecutionHistoryResponse) + err := c.cc.Invoke(ctx, WorkflowService_GetWorkflowExecutionHistory_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) GetWorkflowExecutionHistoryReverse(ctx context.Context, in *GetWorkflowExecutionHistoryReverseRequest, opts ...grpc.CallOption) (*GetWorkflowExecutionHistoryReverseResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetWorkflowExecutionHistoryReverseResponse) + err := c.cc.Invoke(ctx, WorkflowService_GetWorkflowExecutionHistoryReverse_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) PollWorkflowTaskQueue(ctx context.Context, in *PollWorkflowTaskQueueRequest, opts ...grpc.CallOption) (*PollWorkflowTaskQueueResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(PollWorkflowTaskQueueResponse) + err := c.cc.Invoke(ctx, WorkflowService_PollWorkflowTaskQueue_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RespondWorkflowTaskCompleted(ctx context.Context, in *RespondWorkflowTaskCompletedRequest, opts ...grpc.CallOption) (*RespondWorkflowTaskCompletedResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RespondWorkflowTaskCompletedResponse) + err := c.cc.Invoke(ctx, WorkflowService_RespondWorkflowTaskCompleted_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RespondWorkflowTaskFailed(ctx context.Context, in *RespondWorkflowTaskFailedRequest, opts ...grpc.CallOption) (*RespondWorkflowTaskFailedResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RespondWorkflowTaskFailedResponse) + err := c.cc.Invoke(ctx, WorkflowService_RespondWorkflowTaskFailed_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) PollActivityTaskQueue(ctx context.Context, in *PollActivityTaskQueueRequest, opts ...grpc.CallOption) (*PollActivityTaskQueueResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(PollActivityTaskQueueResponse) + err := c.cc.Invoke(ctx, WorkflowService_PollActivityTaskQueue_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RecordActivityTaskHeartbeat(ctx context.Context, in *RecordActivityTaskHeartbeatRequest, opts ...grpc.CallOption) (*RecordActivityTaskHeartbeatResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RecordActivityTaskHeartbeatResponse) + err := c.cc.Invoke(ctx, WorkflowService_RecordActivityTaskHeartbeat_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RecordActivityTaskHeartbeatById(ctx context.Context, in *RecordActivityTaskHeartbeatByIdRequest, opts ...grpc.CallOption) (*RecordActivityTaskHeartbeatByIdResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RecordActivityTaskHeartbeatByIdResponse) + err := c.cc.Invoke(ctx, WorkflowService_RecordActivityTaskHeartbeatById_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RespondActivityTaskCompleted(ctx context.Context, in *RespondActivityTaskCompletedRequest, opts ...grpc.CallOption) (*RespondActivityTaskCompletedResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RespondActivityTaskCompletedResponse) + err := c.cc.Invoke(ctx, WorkflowService_RespondActivityTaskCompleted_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RespondActivityTaskCompletedById(ctx context.Context, in *RespondActivityTaskCompletedByIdRequest, opts ...grpc.CallOption) (*RespondActivityTaskCompletedByIdResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RespondActivityTaskCompletedByIdResponse) + err := c.cc.Invoke(ctx, WorkflowService_RespondActivityTaskCompletedById_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RespondActivityTaskFailed(ctx context.Context, in *RespondActivityTaskFailedRequest, opts ...grpc.CallOption) (*RespondActivityTaskFailedResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RespondActivityTaskFailedResponse) + err := c.cc.Invoke(ctx, WorkflowService_RespondActivityTaskFailed_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RespondActivityTaskFailedById(ctx context.Context, in *RespondActivityTaskFailedByIdRequest, opts ...grpc.CallOption) (*RespondActivityTaskFailedByIdResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RespondActivityTaskFailedByIdResponse) + err := c.cc.Invoke(ctx, WorkflowService_RespondActivityTaskFailedById_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RespondActivityTaskCanceled(ctx context.Context, in *RespondActivityTaskCanceledRequest, opts ...grpc.CallOption) (*RespondActivityTaskCanceledResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RespondActivityTaskCanceledResponse) + err := c.cc.Invoke(ctx, WorkflowService_RespondActivityTaskCanceled_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RespondActivityTaskCanceledById(ctx context.Context, in *RespondActivityTaskCanceledByIdRequest, opts ...grpc.CallOption) (*RespondActivityTaskCanceledByIdResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RespondActivityTaskCanceledByIdResponse) + err := c.cc.Invoke(ctx, WorkflowService_RespondActivityTaskCanceledById_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RequestCancelWorkflowExecution(ctx context.Context, in *RequestCancelWorkflowExecutionRequest, opts ...grpc.CallOption) (*RequestCancelWorkflowExecutionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RequestCancelWorkflowExecutionResponse) + err := c.cc.Invoke(ctx, WorkflowService_RequestCancelWorkflowExecution_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) SignalWorkflowExecution(ctx context.Context, in *SignalWorkflowExecutionRequest, opts ...grpc.CallOption) (*SignalWorkflowExecutionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(SignalWorkflowExecutionResponse) + err := c.cc.Invoke(ctx, WorkflowService_SignalWorkflowExecution_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) SignalWithStartWorkflowExecution(ctx context.Context, in *SignalWithStartWorkflowExecutionRequest, opts ...grpc.CallOption) (*SignalWithStartWorkflowExecutionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(SignalWithStartWorkflowExecutionResponse) + err := c.cc.Invoke(ctx, WorkflowService_SignalWithStartWorkflowExecution_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) ResetWorkflowExecution(ctx context.Context, in *ResetWorkflowExecutionRequest, opts ...grpc.CallOption) (*ResetWorkflowExecutionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResetWorkflowExecutionResponse) + err := c.cc.Invoke(ctx, WorkflowService_ResetWorkflowExecution_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) TerminateWorkflowExecution(ctx context.Context, in *TerminateWorkflowExecutionRequest, opts ...grpc.CallOption) (*TerminateWorkflowExecutionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(TerminateWorkflowExecutionResponse) + err := c.cc.Invoke(ctx, WorkflowService_TerminateWorkflowExecution_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) DeleteWorkflowExecution(ctx context.Context, in *DeleteWorkflowExecutionRequest, opts ...grpc.CallOption) (*DeleteWorkflowExecutionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DeleteWorkflowExecutionResponse) + err := c.cc.Invoke(ctx, WorkflowService_DeleteWorkflowExecution_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) ListOpenWorkflowExecutions(ctx context.Context, in *ListOpenWorkflowExecutionsRequest, opts ...grpc.CallOption) (*ListOpenWorkflowExecutionsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ListOpenWorkflowExecutionsResponse) + err := c.cc.Invoke(ctx, WorkflowService_ListOpenWorkflowExecutions_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) ListClosedWorkflowExecutions(ctx context.Context, in *ListClosedWorkflowExecutionsRequest, opts ...grpc.CallOption) (*ListClosedWorkflowExecutionsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ListClosedWorkflowExecutionsResponse) + err := c.cc.Invoke(ctx, WorkflowService_ListClosedWorkflowExecutions_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) ListWorkflowExecutions(ctx context.Context, in *ListWorkflowExecutionsRequest, opts ...grpc.CallOption) (*ListWorkflowExecutionsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ListWorkflowExecutionsResponse) + err := c.cc.Invoke(ctx, WorkflowService_ListWorkflowExecutions_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) ListArchivedWorkflowExecutions(ctx context.Context, in *ListArchivedWorkflowExecutionsRequest, opts ...grpc.CallOption) (*ListArchivedWorkflowExecutionsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ListArchivedWorkflowExecutionsResponse) + err := c.cc.Invoke(ctx, WorkflowService_ListArchivedWorkflowExecutions_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) ScanWorkflowExecutions(ctx context.Context, in *ScanWorkflowExecutionsRequest, opts ...grpc.CallOption) (*ScanWorkflowExecutionsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ScanWorkflowExecutionsResponse) + err := c.cc.Invoke(ctx, WorkflowService_ScanWorkflowExecutions_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) CountWorkflowExecutions(ctx context.Context, in *CountWorkflowExecutionsRequest, opts ...grpc.CallOption) (*CountWorkflowExecutionsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(CountWorkflowExecutionsResponse) + err := c.cc.Invoke(ctx, WorkflowService_CountWorkflowExecutions_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) GetSearchAttributes(ctx context.Context, in *GetSearchAttributesRequest, opts ...grpc.CallOption) (*GetSearchAttributesResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetSearchAttributesResponse) + err := c.cc.Invoke(ctx, WorkflowService_GetSearchAttributes_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RespondQueryTaskCompleted(ctx context.Context, in *RespondQueryTaskCompletedRequest, opts ...grpc.CallOption) (*RespondQueryTaskCompletedResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RespondQueryTaskCompletedResponse) + err := c.cc.Invoke(ctx, WorkflowService_RespondQueryTaskCompleted_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) ResetStickyTaskQueue(ctx context.Context, in *ResetStickyTaskQueueRequest, opts ...grpc.CallOption) (*ResetStickyTaskQueueResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ResetStickyTaskQueueResponse) + err := c.cc.Invoke(ctx, WorkflowService_ResetStickyTaskQueue_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) QueryWorkflow(ctx context.Context, in *QueryWorkflowRequest, opts ...grpc.CallOption) (*QueryWorkflowResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(QueryWorkflowResponse) + err := c.cc.Invoke(ctx, WorkflowService_QueryWorkflow_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) DescribeWorkflowExecution(ctx context.Context, in *DescribeWorkflowExecutionRequest, opts ...grpc.CallOption) (*DescribeWorkflowExecutionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DescribeWorkflowExecutionResponse) + err := c.cc.Invoke(ctx, WorkflowService_DescribeWorkflowExecution_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) DescribeTaskQueue(ctx context.Context, in *DescribeTaskQueueRequest, opts ...grpc.CallOption) (*DescribeTaskQueueResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DescribeTaskQueueResponse) + err := c.cc.Invoke(ctx, WorkflowService_DescribeTaskQueue_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) GetClusterInfo(ctx context.Context, in *GetClusterInfoRequest, opts ...grpc.CallOption) (*GetClusterInfoResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetClusterInfoResponse) + err := c.cc.Invoke(ctx, WorkflowService_GetClusterInfo_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) GetSystemInfo(ctx context.Context, in *GetSystemInfoRequest, opts ...grpc.CallOption) (*GetSystemInfoResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetSystemInfoResponse) + err := c.cc.Invoke(ctx, WorkflowService_GetSystemInfo_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) ListTaskQueuePartitions(ctx context.Context, in *ListTaskQueuePartitionsRequest, opts ...grpc.CallOption) (*ListTaskQueuePartitionsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ListTaskQueuePartitionsResponse) + err := c.cc.Invoke(ctx, WorkflowService_ListTaskQueuePartitions_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) CreateSchedule(ctx context.Context, in *CreateScheduleRequest, opts ...grpc.CallOption) (*CreateScheduleResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(CreateScheduleResponse) + err := c.cc.Invoke(ctx, WorkflowService_CreateSchedule_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) DescribeSchedule(ctx context.Context, in *DescribeScheduleRequest, opts ...grpc.CallOption) (*DescribeScheduleResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DescribeScheduleResponse) + err := c.cc.Invoke(ctx, WorkflowService_DescribeSchedule_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) UpdateSchedule(ctx context.Context, in *UpdateScheduleRequest, opts ...grpc.CallOption) (*UpdateScheduleResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(UpdateScheduleResponse) + err := c.cc.Invoke(ctx, WorkflowService_UpdateSchedule_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) PatchSchedule(ctx context.Context, in *PatchScheduleRequest, opts ...grpc.CallOption) (*PatchScheduleResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(PatchScheduleResponse) + err := c.cc.Invoke(ctx, WorkflowService_PatchSchedule_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) ListScheduleMatchingTimes(ctx context.Context, in *ListScheduleMatchingTimesRequest, opts ...grpc.CallOption) (*ListScheduleMatchingTimesResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ListScheduleMatchingTimesResponse) + err := c.cc.Invoke(ctx, WorkflowService_ListScheduleMatchingTimes_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) DeleteSchedule(ctx context.Context, in *DeleteScheduleRequest, opts ...grpc.CallOption) (*DeleteScheduleResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DeleteScheduleResponse) + err := c.cc.Invoke(ctx, WorkflowService_DeleteSchedule_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) ListSchedules(ctx context.Context, in *ListSchedulesRequest, opts ...grpc.CallOption) (*ListSchedulesResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ListSchedulesResponse) + err := c.cc.Invoke(ctx, WorkflowService_ListSchedules_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) UpdateWorkerBuildIdCompatibility(ctx context.Context, in *UpdateWorkerBuildIdCompatibilityRequest, opts ...grpc.CallOption) (*UpdateWorkerBuildIdCompatibilityResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(UpdateWorkerBuildIdCompatibilityResponse) + err := c.cc.Invoke(ctx, WorkflowService_UpdateWorkerBuildIdCompatibility_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) GetWorkerBuildIdCompatibility(ctx context.Context, in *GetWorkerBuildIdCompatibilityRequest, opts ...grpc.CallOption) (*GetWorkerBuildIdCompatibilityResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetWorkerBuildIdCompatibilityResponse) + err := c.cc.Invoke(ctx, WorkflowService_GetWorkerBuildIdCompatibility_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) UpdateWorkerVersioningRules(ctx context.Context, in *UpdateWorkerVersioningRulesRequest, opts ...grpc.CallOption) (*UpdateWorkerVersioningRulesResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(UpdateWorkerVersioningRulesResponse) + err := c.cc.Invoke(ctx, WorkflowService_UpdateWorkerVersioningRules_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) GetWorkerVersioningRules(ctx context.Context, in *GetWorkerVersioningRulesRequest, opts ...grpc.CallOption) (*GetWorkerVersioningRulesResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetWorkerVersioningRulesResponse) + err := c.cc.Invoke(ctx, WorkflowService_GetWorkerVersioningRules_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) GetWorkerTaskReachability(ctx context.Context, in *GetWorkerTaskReachabilityRequest, opts ...grpc.CallOption) (*GetWorkerTaskReachabilityResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(GetWorkerTaskReachabilityResponse) + err := c.cc.Invoke(ctx, WorkflowService_GetWorkerTaskReachability_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) UpdateWorkflowExecution(ctx context.Context, in *UpdateWorkflowExecutionRequest, opts ...grpc.CallOption) (*UpdateWorkflowExecutionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(UpdateWorkflowExecutionResponse) + err := c.cc.Invoke(ctx, WorkflowService_UpdateWorkflowExecution_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) PollWorkflowExecutionUpdate(ctx context.Context, in *PollWorkflowExecutionUpdateRequest, opts ...grpc.CallOption) (*PollWorkflowExecutionUpdateResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(PollWorkflowExecutionUpdateResponse) + err := c.cc.Invoke(ctx, WorkflowService_PollWorkflowExecutionUpdate_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) StartBatchOperation(ctx context.Context, in *StartBatchOperationRequest, opts ...grpc.CallOption) (*StartBatchOperationResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(StartBatchOperationResponse) + err := c.cc.Invoke(ctx, WorkflowService_StartBatchOperation_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) StopBatchOperation(ctx context.Context, in *StopBatchOperationRequest, opts ...grpc.CallOption) (*StopBatchOperationResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(StopBatchOperationResponse) + err := c.cc.Invoke(ctx, WorkflowService_StopBatchOperation_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) DescribeBatchOperation(ctx context.Context, in *DescribeBatchOperationRequest, opts ...grpc.CallOption) (*DescribeBatchOperationResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(DescribeBatchOperationResponse) + err := c.cc.Invoke(ctx, WorkflowService_DescribeBatchOperation_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) ListBatchOperations(ctx context.Context, in *ListBatchOperationsRequest, opts ...grpc.CallOption) (*ListBatchOperationsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(ListBatchOperationsResponse) + err := c.cc.Invoke(ctx, WorkflowService_ListBatchOperations_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) PollNexusTaskQueue(ctx context.Context, in *PollNexusTaskQueueRequest, opts ...grpc.CallOption) (*PollNexusTaskQueueResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(PollNexusTaskQueueResponse) + err := c.cc.Invoke(ctx, WorkflowService_PollNexusTaskQueue_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RespondNexusTaskCompleted(ctx context.Context, in *RespondNexusTaskCompletedRequest, opts ...grpc.CallOption) (*RespondNexusTaskCompletedResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RespondNexusTaskCompletedResponse) + err := c.cc.Invoke(ctx, WorkflowService_RespondNexusTaskCompleted_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *workflowServiceClient) RespondNexusTaskFailed(ctx context.Context, in *RespondNexusTaskFailedRequest, opts ...grpc.CallOption) (*RespondNexusTaskFailedResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RespondNexusTaskFailedResponse) + err := c.cc.Invoke(ctx, WorkflowService_RespondNexusTaskFailed_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +// WorkflowServiceServer is the server API for WorkflowService service. +// All implementations must embed UnimplementedWorkflowServiceServer +// for forward compatibility. +// +// WorkflowService API defines how Temporal SDKs and other clients interact with the Temporal server +// to create and interact with workflows and activities. +// +// Users are expected to call `StartWorkflowExecution` to create a new workflow execution. +// +// To drive workflows, a worker using a Temporal SDK must exist which regularly polls for workflow +// and activity tasks from the service. For each workflow task, the sdk must process the +// (incremental or complete) event history and respond back with any newly generated commands. +// +// For each activity task, the worker is expected to execute the user's code which implements that +// activity, responding with completion or failure. +type WorkflowServiceServer interface { + // RegisterNamespace creates a new namespace which can be used as a container for all resources. + // + // A Namespace is a top level entity within Temporal, and is used as a container for resources + // like workflow executions, task queues, etc. A Namespace acts as a sandbox and provides + // isolation for all resources within the namespace. All resources belongs to exactly one + // namespace. + RegisterNamespace(context.Context, *RegisterNamespaceRequest) (*RegisterNamespaceResponse, error) + // DescribeNamespace returns the information and configuration for a registered namespace. + DescribeNamespace(context.Context, *DescribeNamespaceRequest) (*DescribeNamespaceResponse, error) + // ListNamespaces returns the information and configuration for all namespaces. + ListNamespaces(context.Context, *ListNamespacesRequest) (*ListNamespacesResponse, error) + // UpdateNamespace is used to update the information and configuration of a registered + // namespace. + UpdateNamespace(context.Context, *UpdateNamespaceRequest) (*UpdateNamespaceResponse, error) + // DeprecateNamespace is used to update the state of a registered namespace to DEPRECATED. + // + // Once the namespace is deprecated it cannot be used to start new workflow executions. Existing + // workflow executions will continue to run on deprecated namespaces. + // Deprecated. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: Deprecated --) + DeprecateNamespace(context.Context, *DeprecateNamespaceRequest) (*DeprecateNamespaceResponse, error) + // StartWorkflowExecution starts a new workflow execution. + // + // It will create the execution with a `WORKFLOW_EXECUTION_STARTED` event in its history and + // also schedule the first workflow task. Returns `WorkflowExecutionAlreadyStarted`, if an + // instance already exists with same workflow id. + StartWorkflowExecution(context.Context, *StartWorkflowExecutionRequest) (*StartWorkflowExecutionResponse, error) + // ExecuteMultiOperation executes multiple operations within a single workflow. + // + // Operations are started atomically, meaning if *any* operation fails to be started, none are, + // and the request fails. Upon start, the API returns only when *all* operations have a response. + // + // Upon failure, it returns `MultiOperationExecutionFailure` where the status code + // equals the status code of the *first* operation that failed to be started. + // + // NOTE: Experimental API. + ExecuteMultiOperation(context.Context, *ExecuteMultiOperationRequest) (*ExecuteMultiOperationResponse, error) + // GetWorkflowExecutionHistory returns the history of specified workflow execution. Fails with + // `NotFound` if the specified workflow execution is unknown to the service. + GetWorkflowExecutionHistory(context.Context, *GetWorkflowExecutionHistoryRequest) (*GetWorkflowExecutionHistoryResponse, error) + // GetWorkflowExecutionHistoryReverse returns the history of specified workflow execution in reverse + // order (starting from last event). Fails with`NotFound` if the specified workflow execution is + // unknown to the service. + GetWorkflowExecutionHistoryReverse(context.Context, *GetWorkflowExecutionHistoryReverseRequest) (*GetWorkflowExecutionHistoryReverseResponse, error) + // PollWorkflowTaskQueue is called by workers to make progress on workflows. + // + // A WorkflowTask is dispatched to callers for active workflow executions with pending workflow + // tasks. The worker is expected to call `RespondWorkflowTaskCompleted` when it is done + // processing the task. The service will create a `WorkflowTaskStarted` event in the history for + // this task before handing it to the worker. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + PollWorkflowTaskQueue(context.Context, *PollWorkflowTaskQueueRequest) (*PollWorkflowTaskQueueResponse, error) + // RespondWorkflowTaskCompleted is called by workers to successfully complete workflow tasks + // they received from `PollWorkflowTaskQueue`. + // + // Completing a WorkflowTask will write a `WORKFLOW_TASK_COMPLETED` event to the workflow's + // history, along with events corresponding to whatever commands the SDK generated while + // executing the task (ex timer started, activity task scheduled, etc). + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + RespondWorkflowTaskCompleted(context.Context, *RespondWorkflowTaskCompletedRequest) (*RespondWorkflowTaskCompletedResponse, error) + // RespondWorkflowTaskFailed is called by workers to indicate the processing of a workflow task + // failed. + // + // This results in a `WORKFLOW_TASK_FAILED` event written to the history, and a new workflow + // task will be scheduled. This API can be used to report unhandled failures resulting from + // applying the workflow task. + // + // Temporal will only append first WorkflowTaskFailed event to the history of workflow execution + // for consecutive failures. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + RespondWorkflowTaskFailed(context.Context, *RespondWorkflowTaskFailedRequest) (*RespondWorkflowTaskFailedResponse, error) + // PollActivityTaskQueue is called by workers to process activity tasks from a specific task + // queue. + // + // The worker is expected to call one of the `RespondActivityTaskXXX` methods when it is done + // processing the task. + // + // An activity task is dispatched whenever a `SCHEDULE_ACTIVITY_TASK` command is produced during + // workflow execution. An in memory `ACTIVITY_TASK_STARTED` event is written to mutable state + // before the task is dispatched to the worker. The started event, and the final event + // (`ACTIVITY_TASK_COMPLETED` / `ACTIVITY_TASK_FAILED` / `ACTIVITY_TASK_TIMED_OUT`) will both be + // written permanently to Workflow execution history when Activity is finished. This is done to + // avoid writing many events in the case of a failure/retry loop. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + PollActivityTaskQueue(context.Context, *PollActivityTaskQueueRequest) (*PollActivityTaskQueueResponse, error) + // RecordActivityTaskHeartbeat is optionally called by workers while they execute activities. + // + // If worker fails to heartbeat within the `heartbeat_timeout` interval for the activity task, + // then it will be marked as timed out and an `ACTIVITY_TASK_TIMED_OUT` event will be written to + // the workflow history. Calling `RecordActivityTaskHeartbeat` will fail with `NotFound` in + // such situations, in that event, the SDK should request cancellation of the activity. + RecordActivityTaskHeartbeat(context.Context, *RecordActivityTaskHeartbeatRequest) (*RecordActivityTaskHeartbeatResponse, error) + // See `RecordActivityTaskHeartbeat`. This version allows clients to record heartbeats by + // namespace/workflow id/activity id instead of task token. + // + // (-- api-linter: core::0136::prepositions=disabled + // + // aip.dev/not-precedent: "By" is used to indicate request type. --) + RecordActivityTaskHeartbeatById(context.Context, *RecordActivityTaskHeartbeatByIdRequest) (*RecordActivityTaskHeartbeatByIdResponse, error) + // RespondActivityTaskCompleted is called by workers when they successfully complete an activity + // task. + // + // This results in a new `ACTIVITY_TASK_COMPLETED` event being written to the workflow history + // and a new workflow task created for the workflow. Fails with `NotFound` if the task token is + // no longer valid due to activity timeout, already being completed, or never having existed. + RespondActivityTaskCompleted(context.Context, *RespondActivityTaskCompletedRequest) (*RespondActivityTaskCompletedResponse, error) + // See `RecordActivityTaskCompleted`. This version allows clients to record completions by + // namespace/workflow id/activity id instead of task token. + // + // (-- api-linter: core::0136::prepositions=disabled + // + // aip.dev/not-precedent: "By" is used to indicate request type. --) + RespondActivityTaskCompletedById(context.Context, *RespondActivityTaskCompletedByIdRequest) (*RespondActivityTaskCompletedByIdResponse, error) + // RespondActivityTaskFailed is called by workers when processing an activity task fails. + // + // This results in a new `ACTIVITY_TASK_FAILED` event being written to the workflow history and + // a new workflow task created for the workflow. Fails with `NotFound` if the task token is no + // longer valid due to activity timeout, already being completed, or never having existed. + RespondActivityTaskFailed(context.Context, *RespondActivityTaskFailedRequest) (*RespondActivityTaskFailedResponse, error) + // See `RecordActivityTaskFailed`. This version allows clients to record failures by + // namespace/workflow id/activity id instead of task token. + // + // (-- api-linter: core::0136::prepositions=disabled + // + // aip.dev/not-precedent: "By" is used to indicate request type. --) + RespondActivityTaskFailedById(context.Context, *RespondActivityTaskFailedByIdRequest) (*RespondActivityTaskFailedByIdResponse, error) + // RespondActivityTaskFailed is called by workers when processing an activity task fails. + // + // This results in a new `ACTIVITY_TASK_CANCELED` event being written to the workflow history + // and a new workflow task created for the workflow. Fails with `NotFound` if the task token is + // no longer valid due to activity timeout, already being completed, or never having existed. + RespondActivityTaskCanceled(context.Context, *RespondActivityTaskCanceledRequest) (*RespondActivityTaskCanceledResponse, error) + // See `RecordActivityTaskCanceled`. This version allows clients to record failures by + // namespace/workflow id/activity id instead of task token. + // + // (-- api-linter: core::0136::prepositions=disabled + // + // aip.dev/not-precedent: "By" is used to indicate request type. --) + RespondActivityTaskCanceledById(context.Context, *RespondActivityTaskCanceledByIdRequest) (*RespondActivityTaskCanceledByIdResponse, error) + // RequestCancelWorkflowExecution is called by workers when they want to request cancellation of + // a workflow execution. + // + // This results in a new `WORKFLOW_EXECUTION_CANCEL_REQUESTED` event being written to the + // workflow history and a new workflow task created for the workflow. It returns success if the requested + // workflow is already closed. It fails with 'NotFound' if the requested workflow doesn't exist. + RequestCancelWorkflowExecution(context.Context, *RequestCancelWorkflowExecutionRequest) (*RequestCancelWorkflowExecutionResponse, error) + // SignalWorkflowExecution is used to send a signal to a running workflow execution. + // + // This results in a `WORKFLOW_EXECUTION_SIGNALED` event recorded in the history and a workflow + // task being created for the execution. + SignalWorkflowExecution(context.Context, *SignalWorkflowExecutionRequest) (*SignalWorkflowExecutionResponse, error) + // SignalWithStartWorkflowExecution is used to ensure a signal is sent to a workflow, even if + // it isn't yet started. + // + // If the workflow is running, a `WORKFLOW_EXECUTION_SIGNALED` event is recorded in the history + // and a workflow task is generated. + // + // If the workflow is not running or not found, then the workflow is created with + // `WORKFLOW_EXECUTION_STARTED` and `WORKFLOW_EXECUTION_SIGNALED` events in its history, and a + // workflow task is generated. + // + // (-- api-linter: core::0136::prepositions=disabled + // + // aip.dev/not-precedent: "With" is used to indicate combined operation. --) + SignalWithStartWorkflowExecution(context.Context, *SignalWithStartWorkflowExecutionRequest) (*SignalWithStartWorkflowExecutionResponse, error) + // ResetWorkflowExecution will reset an existing workflow execution to a specified + // `WORKFLOW_TASK_COMPLETED` event (exclusive). It will immediately terminate the current + // execution instance. + // TODO: Does exclusive here mean *just* the completed event, or also WFT started? Otherwise the task is doomed to time out? + ResetWorkflowExecution(context.Context, *ResetWorkflowExecutionRequest) (*ResetWorkflowExecutionResponse, error) + // TerminateWorkflowExecution terminates an existing workflow execution by recording a + // `WORKFLOW_EXECUTION_TERMINATED` event in the history and immediately terminating the + // execution instance. + TerminateWorkflowExecution(context.Context, *TerminateWorkflowExecutionRequest) (*TerminateWorkflowExecutionResponse, error) + // DeleteWorkflowExecution asynchronously deletes a specific Workflow Execution (when + // WorkflowExecution.run_id is provided) or the latest Workflow Execution (when + // WorkflowExecution.run_id is not provided). If the Workflow Execution is Running, it will be + // terminated before deletion. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: Workflow deletion not exposed to HTTP, users should use cancel or terminate. --) + DeleteWorkflowExecution(context.Context, *DeleteWorkflowExecutionRequest) (*DeleteWorkflowExecutionResponse, error) + // ListOpenWorkflowExecutions is a visibility API to list the open executions in a specific namespace. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: HTTP users should use ListWorkflowExecutions instead. --) + ListOpenWorkflowExecutions(context.Context, *ListOpenWorkflowExecutionsRequest) (*ListOpenWorkflowExecutionsResponse, error) + // ListClosedWorkflowExecutions is a visibility API to list the closed executions in a specific namespace. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: HTTP users should use ListWorkflowExecutions instead. --) + ListClosedWorkflowExecutions(context.Context, *ListClosedWorkflowExecutionsRequest) (*ListClosedWorkflowExecutionsResponse, error) + // ListWorkflowExecutions is a visibility API to list workflow executions in a specific namespace. + ListWorkflowExecutions(context.Context, *ListWorkflowExecutionsRequest) (*ListWorkflowExecutionsResponse, error) + // ListArchivedWorkflowExecutions is a visibility API to list archived workflow executions in a specific namespace. + ListArchivedWorkflowExecutions(context.Context, *ListArchivedWorkflowExecutionsRequest) (*ListArchivedWorkflowExecutionsResponse, error) + // ScanWorkflowExecutions is a visibility API to list large amount of workflow executions in a specific namespace without order. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: HTTP users should use ListWorkflowExecutions instead. --) + ScanWorkflowExecutions(context.Context, *ScanWorkflowExecutionsRequest) (*ScanWorkflowExecutionsResponse, error) + // CountWorkflowExecutions is a visibility API to count of workflow executions in a specific namespace. + CountWorkflowExecutions(context.Context, *CountWorkflowExecutionsRequest) (*CountWorkflowExecutionsResponse, error) + // GetSearchAttributes is a visibility API to get all legal keys that could be used in list APIs + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose this search attribute API to HTTP (but may expose on OperatorService). --) + GetSearchAttributes(context.Context, *GetSearchAttributesRequest) (*GetSearchAttributesResponse, error) + // RespondQueryTaskCompleted is called by workers to complete queries which were delivered on + // the `query` (not `queries`) field of a `PollWorkflowTaskQueueResponse`. + // + // Completing the query will unblock the corresponding client call to `QueryWorkflow` and return + // the query result a response. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + RespondQueryTaskCompleted(context.Context, *RespondQueryTaskCompletedRequest) (*RespondQueryTaskCompletedResponse, error) + // ResetStickyTaskQueue resets the sticky task queue related information in the mutable state of + // a given workflow. This is prudent for workers to perform if a workflow has been paged out of + // their cache. + // + // Things cleared are: + // 1. StickyTaskQueue + // 2. StickyScheduleToStartTimeout + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + ResetStickyTaskQueue(context.Context, *ResetStickyTaskQueueRequest) (*ResetStickyTaskQueueResponse, error) + // QueryWorkflow requests a query be executed for a specified workflow execution. + QueryWorkflow(context.Context, *QueryWorkflowRequest) (*QueryWorkflowResponse, error) + // DescribeWorkflowExecution returns information about the specified workflow execution. + DescribeWorkflowExecution(context.Context, *DescribeWorkflowExecutionRequest) (*DescribeWorkflowExecutionResponse, error) + // DescribeTaskQueue returns the following information about the target task queue, broken down by Build ID: + // - List of pollers + // - Workflow Reachability status + // - Backlog info for Workflow and/or Activity tasks + DescribeTaskQueue(context.Context, *DescribeTaskQueueRequest) (*DescribeTaskQueueResponse, error) + // GetClusterInfo returns information about temporal cluster + GetClusterInfo(context.Context, *GetClusterInfoRequest) (*GetClusterInfoResponse, error) + // GetSystemInfo returns information about the system. + GetSystemInfo(context.Context, *GetSystemInfoRequest) (*GetSystemInfoResponse, error) + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose this low-level API to HTTP. --) + ListTaskQueuePartitions(context.Context, *ListTaskQueuePartitionsRequest) (*ListTaskQueuePartitionsResponse, error) + // Creates a new schedule. + CreateSchedule(context.Context, *CreateScheduleRequest) (*CreateScheduleResponse, error) + // Returns the schedule description and current state of an existing schedule. + DescribeSchedule(context.Context, *DescribeScheduleRequest) (*DescribeScheduleResponse, error) + // Changes the configuration or state of an existing schedule. + UpdateSchedule(context.Context, *UpdateScheduleRequest) (*UpdateScheduleResponse, error) + // Makes a specific change to a schedule or triggers an immediate action. + PatchSchedule(context.Context, *PatchScheduleRequest) (*PatchScheduleResponse, error) + // Lists matching times within a range. + ListScheduleMatchingTimes(context.Context, *ListScheduleMatchingTimesRequest) (*ListScheduleMatchingTimesResponse, error) + // Deletes a schedule, removing it from the system. + DeleteSchedule(context.Context, *DeleteScheduleRequest) (*DeleteScheduleResponse, error) + // List all schedules in a namespace. + ListSchedules(context.Context, *ListSchedulesRequest) (*ListSchedulesResponse, error) + // Deprecated. Use `UpdateWorkerVersioningRules`. + // + // Allows users to specify sets of worker build id versions on a per task queue basis. Versions + // are ordered, and may be either compatible with some extant version, or a new incompatible + // version, forming sets of ids which are incompatible with each other, but whose contained + // members are compatible with one another. + // + // A single build id may be mapped to multiple task queues using this API for cases where a single process hosts + // multiple workers. + // + // To query which workers can be retired, use the `GetWorkerTaskReachability` API. + // + // NOTE: The number of task queues mapped to a single build id is limited by the `limit.taskQueuesPerBuildId` + // (default is 20), if this limit is exceeded this API will error with a FailedPrecondition. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do yet expose versioning API to HTTP. --) + UpdateWorkerBuildIdCompatibility(context.Context, *UpdateWorkerBuildIdCompatibilityRequest) (*UpdateWorkerBuildIdCompatibilityResponse, error) + // Deprecated. Use `GetWorkerVersioningRules`. + // Fetches the worker build id versioning sets for a task queue. + GetWorkerBuildIdCompatibility(context.Context, *GetWorkerBuildIdCompatibilityRequest) (*GetWorkerBuildIdCompatibilityResponse, error) + // Use this API to manage Worker Versioning Rules for a given Task Queue. There are two types of + // rules: Build ID Assignment rules and Compatible Build ID Redirect rules. + // + // Assignment rules determine how to assign new executions to a Build IDs. Their primary + // use case is to specify the latest Build ID but they have powerful features for gradual rollout + // of a new Build ID. + // + // Once a workflow execution is assigned to a Build ID and it completes its first Workflow Task, + // the workflow stays on the assigned Build ID regardless of changes in assignment rules. This + // eliminates the need for compatibility between versions when you only care about using the new + // version for new workflows and let existing workflows finish in their own version. + // + // Activities, Child Workflows and Continue-as-New executions have the option to inherit the + // Build ID of their parent/previous workflow or use the latest assignment rules to independently + // select a Build ID. + // + // Redirect rules should only be used when you want to move workflows and activities assigned to + // one Build ID (source) to another compatible Build ID (target). You are responsible to make sure + // the target Build ID of a redirect rule is able to process event histories made by the source + // Build ID by using [Patching](https://docs.temporal.io/workflows#patching) or other means. + // + // WARNING: Worker Versioning is not yet stable and the API and behavior may change incompatibly. + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do yet expose versioning API to HTTP. --) + UpdateWorkerVersioningRules(context.Context, *UpdateWorkerVersioningRulesRequest) (*UpdateWorkerVersioningRulesResponse, error) + // Fetches the Build ID assignment and redirect rules for a Task Queue. + // WARNING: Worker Versioning is not yet stable and the API and behavior may change incompatibly. + GetWorkerVersioningRules(context.Context, *GetWorkerVersioningRulesRequest) (*GetWorkerVersioningRulesResponse, error) + // Deprecated. Use `DescribeTaskQueue`. + // + // Fetches task reachability to determine whether a worker may be retired. + // The request may specify task queues to query for or let the server fetch all task queues mapped to the given + // build IDs. + // + // When requesting a large number of task queues or all task queues associated with the given build ids in a + // namespace, all task queues will be listed in the response but some of them may not contain reachability + // information due to a server enforced limit. When reaching the limit, task queues that reachability information + // could not be retrieved for will be marked with a single TASK_REACHABILITY_UNSPECIFIED entry. The caller may issue + // another call to get the reachability for those task queues. + // + // Open source users can adjust this limit by setting the server's dynamic config value for + // `limit.reachabilityTaskQueueScan` with the caveat that this call can strain the visibility store. + GetWorkerTaskReachability(context.Context, *GetWorkerTaskReachabilityRequest) (*GetWorkerTaskReachabilityResponse, error) + // Invokes the specified update function on user workflow code. + UpdateWorkflowExecution(context.Context, *UpdateWorkflowExecutionRequest) (*UpdateWorkflowExecutionResponse, error) + // Polls a workflow execution for the outcome of a workflow execution update + // previously issued through the UpdateWorkflowExecution RPC. The effective + // timeout on this call will be shorter of the the caller-supplied gRPC + // timeout and the server's configured long-poll timeout. + // + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We don't expose update polling API to HTTP in favor of a potential future non-blocking form. --) + PollWorkflowExecutionUpdate(context.Context, *PollWorkflowExecutionUpdateRequest) (*PollWorkflowExecutionUpdateResponse, error) + // StartBatchOperation starts a new batch operation + StartBatchOperation(context.Context, *StartBatchOperationRequest) (*StartBatchOperationResponse, error) + // StopBatchOperation stops a batch operation + StopBatchOperation(context.Context, *StopBatchOperationRequest) (*StopBatchOperationResponse, error) + // DescribeBatchOperation returns the information about a batch operation + DescribeBatchOperation(context.Context, *DescribeBatchOperationRequest) (*DescribeBatchOperationResponse, error) + // ListBatchOperations returns a list of batch operations + ListBatchOperations(context.Context, *ListBatchOperationsRequest) (*ListBatchOperationsResponse, error) + // PollNexusTaskQueue is a long poll call used by workers to receive Nexus tasks. + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + PollNexusTaskQueue(context.Context, *PollNexusTaskQueueRequest) (*PollNexusTaskQueueResponse, error) + // RespondNexusTaskCompleted is called by workers to respond to Nexus tasks received via PollNexusTaskQueue. + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + RespondNexusTaskCompleted(context.Context, *RespondNexusTaskCompletedRequest) (*RespondNexusTaskCompletedResponse, error) + // RespondNexusTaskFailed is called by workers to fail Nexus tasks received via PollNexusTaskQueue. + // (-- api-linter: core::0127::http-annotation=disabled + // + // aip.dev/not-precedent: We do not expose worker API to HTTP. --) + RespondNexusTaskFailed(context.Context, *RespondNexusTaskFailedRequest) (*RespondNexusTaskFailedResponse, error) + mustEmbedUnimplementedWorkflowServiceServer() +} + +// UnimplementedWorkflowServiceServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedWorkflowServiceServer struct{} + +func (UnimplementedWorkflowServiceServer) RegisterNamespace(context.Context, *RegisterNamespaceRequest) (*RegisterNamespaceResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RegisterNamespace not implemented") +} +func (UnimplementedWorkflowServiceServer) DescribeNamespace(context.Context, *DescribeNamespaceRequest) (*DescribeNamespaceResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DescribeNamespace not implemented") +} +func (UnimplementedWorkflowServiceServer) ListNamespaces(context.Context, *ListNamespacesRequest) (*ListNamespacesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListNamespaces not implemented") +} +func (UnimplementedWorkflowServiceServer) UpdateNamespace(context.Context, *UpdateNamespaceRequest) (*UpdateNamespaceResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateNamespace not implemented") +} +func (UnimplementedWorkflowServiceServer) DeprecateNamespace(context.Context, *DeprecateNamespaceRequest) (*DeprecateNamespaceResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeprecateNamespace not implemented") +} +func (UnimplementedWorkflowServiceServer) StartWorkflowExecution(context.Context, *StartWorkflowExecutionRequest) (*StartWorkflowExecutionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method StartWorkflowExecution not implemented") +} +func (UnimplementedWorkflowServiceServer) ExecuteMultiOperation(context.Context, *ExecuteMultiOperationRequest) (*ExecuteMultiOperationResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ExecuteMultiOperation not implemented") +} +func (UnimplementedWorkflowServiceServer) GetWorkflowExecutionHistory(context.Context, *GetWorkflowExecutionHistoryRequest) (*GetWorkflowExecutionHistoryResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetWorkflowExecutionHistory not implemented") +} +func (UnimplementedWorkflowServiceServer) GetWorkflowExecutionHistoryReverse(context.Context, *GetWorkflowExecutionHistoryReverseRequest) (*GetWorkflowExecutionHistoryReverseResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetWorkflowExecutionHistoryReverse not implemented") +} +func (UnimplementedWorkflowServiceServer) PollWorkflowTaskQueue(context.Context, *PollWorkflowTaskQueueRequest) (*PollWorkflowTaskQueueResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method PollWorkflowTaskQueue not implemented") +} +func (UnimplementedWorkflowServiceServer) RespondWorkflowTaskCompleted(context.Context, *RespondWorkflowTaskCompletedRequest) (*RespondWorkflowTaskCompletedResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RespondWorkflowTaskCompleted not implemented") +} +func (UnimplementedWorkflowServiceServer) RespondWorkflowTaskFailed(context.Context, *RespondWorkflowTaskFailedRequest) (*RespondWorkflowTaskFailedResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RespondWorkflowTaskFailed not implemented") +} +func (UnimplementedWorkflowServiceServer) PollActivityTaskQueue(context.Context, *PollActivityTaskQueueRequest) (*PollActivityTaskQueueResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method PollActivityTaskQueue not implemented") +} +func (UnimplementedWorkflowServiceServer) RecordActivityTaskHeartbeat(context.Context, *RecordActivityTaskHeartbeatRequest) (*RecordActivityTaskHeartbeatResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RecordActivityTaskHeartbeat not implemented") +} +func (UnimplementedWorkflowServiceServer) RecordActivityTaskHeartbeatById(context.Context, *RecordActivityTaskHeartbeatByIdRequest) (*RecordActivityTaskHeartbeatByIdResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RecordActivityTaskHeartbeatById not implemented") +} +func (UnimplementedWorkflowServiceServer) RespondActivityTaskCompleted(context.Context, *RespondActivityTaskCompletedRequest) (*RespondActivityTaskCompletedResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RespondActivityTaskCompleted not implemented") +} +func (UnimplementedWorkflowServiceServer) RespondActivityTaskCompletedById(context.Context, *RespondActivityTaskCompletedByIdRequest) (*RespondActivityTaskCompletedByIdResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RespondActivityTaskCompletedById not implemented") +} +func (UnimplementedWorkflowServiceServer) RespondActivityTaskFailed(context.Context, *RespondActivityTaskFailedRequest) (*RespondActivityTaskFailedResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RespondActivityTaskFailed not implemented") +} +func (UnimplementedWorkflowServiceServer) RespondActivityTaskFailedById(context.Context, *RespondActivityTaskFailedByIdRequest) (*RespondActivityTaskFailedByIdResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RespondActivityTaskFailedById not implemented") +} +func (UnimplementedWorkflowServiceServer) RespondActivityTaskCanceled(context.Context, *RespondActivityTaskCanceledRequest) (*RespondActivityTaskCanceledResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RespondActivityTaskCanceled not implemented") +} +func (UnimplementedWorkflowServiceServer) RespondActivityTaskCanceledById(context.Context, *RespondActivityTaskCanceledByIdRequest) (*RespondActivityTaskCanceledByIdResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RespondActivityTaskCanceledById not implemented") +} +func (UnimplementedWorkflowServiceServer) RequestCancelWorkflowExecution(context.Context, *RequestCancelWorkflowExecutionRequest) (*RequestCancelWorkflowExecutionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RequestCancelWorkflowExecution not implemented") +} +func (UnimplementedWorkflowServiceServer) SignalWorkflowExecution(context.Context, *SignalWorkflowExecutionRequest) (*SignalWorkflowExecutionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SignalWorkflowExecution not implemented") +} +func (UnimplementedWorkflowServiceServer) SignalWithStartWorkflowExecution(context.Context, *SignalWithStartWorkflowExecutionRequest) (*SignalWithStartWorkflowExecutionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SignalWithStartWorkflowExecution not implemented") +} +func (UnimplementedWorkflowServiceServer) ResetWorkflowExecution(context.Context, *ResetWorkflowExecutionRequest) (*ResetWorkflowExecutionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResetWorkflowExecution not implemented") +} +func (UnimplementedWorkflowServiceServer) TerminateWorkflowExecution(context.Context, *TerminateWorkflowExecutionRequest) (*TerminateWorkflowExecutionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method TerminateWorkflowExecution not implemented") +} +func (UnimplementedWorkflowServiceServer) DeleteWorkflowExecution(context.Context, *DeleteWorkflowExecutionRequest) (*DeleteWorkflowExecutionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteWorkflowExecution not implemented") +} +func (UnimplementedWorkflowServiceServer) ListOpenWorkflowExecutions(context.Context, *ListOpenWorkflowExecutionsRequest) (*ListOpenWorkflowExecutionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListOpenWorkflowExecutions not implemented") +} +func (UnimplementedWorkflowServiceServer) ListClosedWorkflowExecutions(context.Context, *ListClosedWorkflowExecutionsRequest) (*ListClosedWorkflowExecutionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListClosedWorkflowExecutions not implemented") +} +func (UnimplementedWorkflowServiceServer) ListWorkflowExecutions(context.Context, *ListWorkflowExecutionsRequest) (*ListWorkflowExecutionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListWorkflowExecutions not implemented") +} +func (UnimplementedWorkflowServiceServer) ListArchivedWorkflowExecutions(context.Context, *ListArchivedWorkflowExecutionsRequest) (*ListArchivedWorkflowExecutionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListArchivedWorkflowExecutions not implemented") +} +func (UnimplementedWorkflowServiceServer) ScanWorkflowExecutions(context.Context, *ScanWorkflowExecutionsRequest) (*ScanWorkflowExecutionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ScanWorkflowExecutions not implemented") +} +func (UnimplementedWorkflowServiceServer) CountWorkflowExecutions(context.Context, *CountWorkflowExecutionsRequest) (*CountWorkflowExecutionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CountWorkflowExecutions not implemented") +} +func (UnimplementedWorkflowServiceServer) GetSearchAttributes(context.Context, *GetSearchAttributesRequest) (*GetSearchAttributesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetSearchAttributes not implemented") +} +func (UnimplementedWorkflowServiceServer) RespondQueryTaskCompleted(context.Context, *RespondQueryTaskCompletedRequest) (*RespondQueryTaskCompletedResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RespondQueryTaskCompleted not implemented") +} +func (UnimplementedWorkflowServiceServer) ResetStickyTaskQueue(context.Context, *ResetStickyTaskQueueRequest) (*ResetStickyTaskQueueResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResetStickyTaskQueue not implemented") +} +func (UnimplementedWorkflowServiceServer) QueryWorkflow(context.Context, *QueryWorkflowRequest) (*QueryWorkflowResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method QueryWorkflow not implemented") +} +func (UnimplementedWorkflowServiceServer) DescribeWorkflowExecution(context.Context, *DescribeWorkflowExecutionRequest) (*DescribeWorkflowExecutionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DescribeWorkflowExecution not implemented") +} +func (UnimplementedWorkflowServiceServer) DescribeTaskQueue(context.Context, *DescribeTaskQueueRequest) (*DescribeTaskQueueResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DescribeTaskQueue not implemented") +} +func (UnimplementedWorkflowServiceServer) GetClusterInfo(context.Context, *GetClusterInfoRequest) (*GetClusterInfoResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetClusterInfo not implemented") +} +func (UnimplementedWorkflowServiceServer) GetSystemInfo(context.Context, *GetSystemInfoRequest) (*GetSystemInfoResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetSystemInfo not implemented") +} +func (UnimplementedWorkflowServiceServer) ListTaskQueuePartitions(context.Context, *ListTaskQueuePartitionsRequest) (*ListTaskQueuePartitionsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListTaskQueuePartitions not implemented") +} +func (UnimplementedWorkflowServiceServer) CreateSchedule(context.Context, *CreateScheduleRequest) (*CreateScheduleResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateSchedule not implemented") +} +func (UnimplementedWorkflowServiceServer) DescribeSchedule(context.Context, *DescribeScheduleRequest) (*DescribeScheduleResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DescribeSchedule not implemented") +} +func (UnimplementedWorkflowServiceServer) UpdateSchedule(context.Context, *UpdateScheduleRequest) (*UpdateScheduleResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateSchedule not implemented") +} +func (UnimplementedWorkflowServiceServer) PatchSchedule(context.Context, *PatchScheduleRequest) (*PatchScheduleResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method PatchSchedule not implemented") +} +func (UnimplementedWorkflowServiceServer) ListScheduleMatchingTimes(context.Context, *ListScheduleMatchingTimesRequest) (*ListScheduleMatchingTimesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListScheduleMatchingTimes not implemented") +} +func (UnimplementedWorkflowServiceServer) DeleteSchedule(context.Context, *DeleteScheduleRequest) (*DeleteScheduleResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteSchedule not implemented") +} +func (UnimplementedWorkflowServiceServer) ListSchedules(context.Context, *ListSchedulesRequest) (*ListSchedulesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListSchedules not implemented") +} +func (UnimplementedWorkflowServiceServer) UpdateWorkerBuildIdCompatibility(context.Context, *UpdateWorkerBuildIdCompatibilityRequest) (*UpdateWorkerBuildIdCompatibilityResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateWorkerBuildIdCompatibility not implemented") +} +func (UnimplementedWorkflowServiceServer) GetWorkerBuildIdCompatibility(context.Context, *GetWorkerBuildIdCompatibilityRequest) (*GetWorkerBuildIdCompatibilityResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetWorkerBuildIdCompatibility not implemented") +} +func (UnimplementedWorkflowServiceServer) UpdateWorkerVersioningRules(context.Context, *UpdateWorkerVersioningRulesRequest) (*UpdateWorkerVersioningRulesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateWorkerVersioningRules not implemented") +} +func (UnimplementedWorkflowServiceServer) GetWorkerVersioningRules(context.Context, *GetWorkerVersioningRulesRequest) (*GetWorkerVersioningRulesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetWorkerVersioningRules not implemented") +} +func (UnimplementedWorkflowServiceServer) GetWorkerTaskReachability(context.Context, *GetWorkerTaskReachabilityRequest) (*GetWorkerTaskReachabilityResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetWorkerTaskReachability not implemented") +} +func (UnimplementedWorkflowServiceServer) UpdateWorkflowExecution(context.Context, *UpdateWorkflowExecutionRequest) (*UpdateWorkflowExecutionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateWorkflowExecution not implemented") +} +func (UnimplementedWorkflowServiceServer) PollWorkflowExecutionUpdate(context.Context, *PollWorkflowExecutionUpdateRequest) (*PollWorkflowExecutionUpdateResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method PollWorkflowExecutionUpdate not implemented") +} +func (UnimplementedWorkflowServiceServer) StartBatchOperation(context.Context, *StartBatchOperationRequest) (*StartBatchOperationResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method StartBatchOperation not implemented") +} +func (UnimplementedWorkflowServiceServer) StopBatchOperation(context.Context, *StopBatchOperationRequest) (*StopBatchOperationResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method StopBatchOperation not implemented") +} +func (UnimplementedWorkflowServiceServer) DescribeBatchOperation(context.Context, *DescribeBatchOperationRequest) (*DescribeBatchOperationResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method DescribeBatchOperation not implemented") +} +func (UnimplementedWorkflowServiceServer) ListBatchOperations(context.Context, *ListBatchOperationsRequest) (*ListBatchOperationsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListBatchOperations not implemented") +} +func (UnimplementedWorkflowServiceServer) PollNexusTaskQueue(context.Context, *PollNexusTaskQueueRequest) (*PollNexusTaskQueueResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method PollNexusTaskQueue not implemented") +} +func (UnimplementedWorkflowServiceServer) RespondNexusTaskCompleted(context.Context, *RespondNexusTaskCompletedRequest) (*RespondNexusTaskCompletedResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RespondNexusTaskCompleted not implemented") +} +func (UnimplementedWorkflowServiceServer) RespondNexusTaskFailed(context.Context, *RespondNexusTaskFailedRequest) (*RespondNexusTaskFailedResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RespondNexusTaskFailed not implemented") +} +func (UnimplementedWorkflowServiceServer) mustEmbedUnimplementedWorkflowServiceServer() {} +func (UnimplementedWorkflowServiceServer) testEmbeddedByValue() {} + +// UnsafeWorkflowServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to WorkflowServiceServer will +// result in compilation errors. +type UnsafeWorkflowServiceServer interface { + mustEmbedUnimplementedWorkflowServiceServer() +} + +func RegisterWorkflowServiceServer(s grpc.ServiceRegistrar, srv WorkflowServiceServer) { + // If the following call pancis, it indicates UnimplementedWorkflowServiceServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } + s.RegisterService(&WorkflowService_ServiceDesc, srv) +} + +func _WorkflowService_RegisterNamespace_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RegisterNamespaceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RegisterNamespace(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RegisterNamespace_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RegisterNamespace(ctx, req.(*RegisterNamespaceRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_DescribeNamespace_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DescribeNamespaceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).DescribeNamespace(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_DescribeNamespace_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).DescribeNamespace(ctx, req.(*DescribeNamespaceRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_ListNamespaces_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListNamespacesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).ListNamespaces(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_ListNamespaces_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).ListNamespaces(ctx, req.(*ListNamespacesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_UpdateNamespace_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateNamespaceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).UpdateNamespace(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_UpdateNamespace_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).UpdateNamespace(ctx, req.(*UpdateNamespaceRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_DeprecateNamespace_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeprecateNamespaceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).DeprecateNamespace(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_DeprecateNamespace_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).DeprecateNamespace(ctx, req.(*DeprecateNamespaceRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_StartWorkflowExecution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(StartWorkflowExecutionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).StartWorkflowExecution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_StartWorkflowExecution_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).StartWorkflowExecution(ctx, req.(*StartWorkflowExecutionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_ExecuteMultiOperation_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ExecuteMultiOperationRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).ExecuteMultiOperation(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_ExecuteMultiOperation_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).ExecuteMultiOperation(ctx, req.(*ExecuteMultiOperationRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_GetWorkflowExecutionHistory_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetWorkflowExecutionHistoryRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).GetWorkflowExecutionHistory(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_GetWorkflowExecutionHistory_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).GetWorkflowExecutionHistory(ctx, req.(*GetWorkflowExecutionHistoryRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_GetWorkflowExecutionHistoryReverse_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetWorkflowExecutionHistoryReverseRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).GetWorkflowExecutionHistoryReverse(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_GetWorkflowExecutionHistoryReverse_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).GetWorkflowExecutionHistoryReverse(ctx, req.(*GetWorkflowExecutionHistoryReverseRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_PollWorkflowTaskQueue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(PollWorkflowTaskQueueRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).PollWorkflowTaskQueue(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_PollWorkflowTaskQueue_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).PollWorkflowTaskQueue(ctx, req.(*PollWorkflowTaskQueueRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RespondWorkflowTaskCompleted_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RespondWorkflowTaskCompletedRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RespondWorkflowTaskCompleted(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RespondWorkflowTaskCompleted_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RespondWorkflowTaskCompleted(ctx, req.(*RespondWorkflowTaskCompletedRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RespondWorkflowTaskFailed_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RespondWorkflowTaskFailedRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RespondWorkflowTaskFailed(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RespondWorkflowTaskFailed_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RespondWorkflowTaskFailed(ctx, req.(*RespondWorkflowTaskFailedRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_PollActivityTaskQueue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(PollActivityTaskQueueRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).PollActivityTaskQueue(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_PollActivityTaskQueue_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).PollActivityTaskQueue(ctx, req.(*PollActivityTaskQueueRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RecordActivityTaskHeartbeat_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RecordActivityTaskHeartbeatRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RecordActivityTaskHeartbeat(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RecordActivityTaskHeartbeat_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RecordActivityTaskHeartbeat(ctx, req.(*RecordActivityTaskHeartbeatRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RecordActivityTaskHeartbeatById_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RecordActivityTaskHeartbeatByIdRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RecordActivityTaskHeartbeatById(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RecordActivityTaskHeartbeatById_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RecordActivityTaskHeartbeatById(ctx, req.(*RecordActivityTaskHeartbeatByIdRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RespondActivityTaskCompleted_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RespondActivityTaskCompletedRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RespondActivityTaskCompleted(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RespondActivityTaskCompleted_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RespondActivityTaskCompleted(ctx, req.(*RespondActivityTaskCompletedRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RespondActivityTaskCompletedById_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RespondActivityTaskCompletedByIdRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RespondActivityTaskCompletedById(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RespondActivityTaskCompletedById_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RespondActivityTaskCompletedById(ctx, req.(*RespondActivityTaskCompletedByIdRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RespondActivityTaskFailed_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RespondActivityTaskFailedRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RespondActivityTaskFailed(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RespondActivityTaskFailed_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RespondActivityTaskFailed(ctx, req.(*RespondActivityTaskFailedRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RespondActivityTaskFailedById_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RespondActivityTaskFailedByIdRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RespondActivityTaskFailedById(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RespondActivityTaskFailedById_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RespondActivityTaskFailedById(ctx, req.(*RespondActivityTaskFailedByIdRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RespondActivityTaskCanceled_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RespondActivityTaskCanceledRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RespondActivityTaskCanceled(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RespondActivityTaskCanceled_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RespondActivityTaskCanceled(ctx, req.(*RespondActivityTaskCanceledRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RespondActivityTaskCanceledById_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RespondActivityTaskCanceledByIdRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RespondActivityTaskCanceledById(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RespondActivityTaskCanceledById_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RespondActivityTaskCanceledById(ctx, req.(*RespondActivityTaskCanceledByIdRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RequestCancelWorkflowExecution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RequestCancelWorkflowExecutionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RequestCancelWorkflowExecution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RequestCancelWorkflowExecution_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RequestCancelWorkflowExecution(ctx, req.(*RequestCancelWorkflowExecutionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_SignalWorkflowExecution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SignalWorkflowExecutionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).SignalWorkflowExecution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_SignalWorkflowExecution_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).SignalWorkflowExecution(ctx, req.(*SignalWorkflowExecutionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_SignalWithStartWorkflowExecution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SignalWithStartWorkflowExecutionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).SignalWithStartWorkflowExecution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_SignalWithStartWorkflowExecution_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).SignalWithStartWorkflowExecution(ctx, req.(*SignalWithStartWorkflowExecutionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_ResetWorkflowExecution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResetWorkflowExecutionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).ResetWorkflowExecution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_ResetWorkflowExecution_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).ResetWorkflowExecution(ctx, req.(*ResetWorkflowExecutionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_TerminateWorkflowExecution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(TerminateWorkflowExecutionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).TerminateWorkflowExecution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_TerminateWorkflowExecution_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).TerminateWorkflowExecution(ctx, req.(*TerminateWorkflowExecutionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_DeleteWorkflowExecution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteWorkflowExecutionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).DeleteWorkflowExecution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_DeleteWorkflowExecution_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).DeleteWorkflowExecution(ctx, req.(*DeleteWorkflowExecutionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_ListOpenWorkflowExecutions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListOpenWorkflowExecutionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).ListOpenWorkflowExecutions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_ListOpenWorkflowExecutions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).ListOpenWorkflowExecutions(ctx, req.(*ListOpenWorkflowExecutionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_ListClosedWorkflowExecutions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListClosedWorkflowExecutionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).ListClosedWorkflowExecutions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_ListClosedWorkflowExecutions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).ListClosedWorkflowExecutions(ctx, req.(*ListClosedWorkflowExecutionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_ListWorkflowExecutions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListWorkflowExecutionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).ListWorkflowExecutions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_ListWorkflowExecutions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).ListWorkflowExecutions(ctx, req.(*ListWorkflowExecutionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_ListArchivedWorkflowExecutions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListArchivedWorkflowExecutionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).ListArchivedWorkflowExecutions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_ListArchivedWorkflowExecutions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).ListArchivedWorkflowExecutions(ctx, req.(*ListArchivedWorkflowExecutionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_ScanWorkflowExecutions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ScanWorkflowExecutionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).ScanWorkflowExecutions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_ScanWorkflowExecutions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).ScanWorkflowExecutions(ctx, req.(*ScanWorkflowExecutionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_CountWorkflowExecutions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CountWorkflowExecutionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).CountWorkflowExecutions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_CountWorkflowExecutions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).CountWorkflowExecutions(ctx, req.(*CountWorkflowExecutionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_GetSearchAttributes_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetSearchAttributesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).GetSearchAttributes(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_GetSearchAttributes_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).GetSearchAttributes(ctx, req.(*GetSearchAttributesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RespondQueryTaskCompleted_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RespondQueryTaskCompletedRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RespondQueryTaskCompleted(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RespondQueryTaskCompleted_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RespondQueryTaskCompleted(ctx, req.(*RespondQueryTaskCompletedRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_ResetStickyTaskQueue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResetStickyTaskQueueRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).ResetStickyTaskQueue(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_ResetStickyTaskQueue_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).ResetStickyTaskQueue(ctx, req.(*ResetStickyTaskQueueRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_QueryWorkflow_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(QueryWorkflowRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).QueryWorkflow(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_QueryWorkflow_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).QueryWorkflow(ctx, req.(*QueryWorkflowRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_DescribeWorkflowExecution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DescribeWorkflowExecutionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).DescribeWorkflowExecution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_DescribeWorkflowExecution_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).DescribeWorkflowExecution(ctx, req.(*DescribeWorkflowExecutionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_DescribeTaskQueue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DescribeTaskQueueRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).DescribeTaskQueue(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_DescribeTaskQueue_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).DescribeTaskQueue(ctx, req.(*DescribeTaskQueueRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_GetClusterInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetClusterInfoRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).GetClusterInfo(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_GetClusterInfo_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).GetClusterInfo(ctx, req.(*GetClusterInfoRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_GetSystemInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetSystemInfoRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).GetSystemInfo(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_GetSystemInfo_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).GetSystemInfo(ctx, req.(*GetSystemInfoRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_ListTaskQueuePartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListTaskQueuePartitionsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).ListTaskQueuePartitions(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_ListTaskQueuePartitions_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).ListTaskQueuePartitions(ctx, req.(*ListTaskQueuePartitionsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_CreateSchedule_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateScheduleRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).CreateSchedule(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_CreateSchedule_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).CreateSchedule(ctx, req.(*CreateScheduleRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_DescribeSchedule_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DescribeScheduleRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).DescribeSchedule(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_DescribeSchedule_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).DescribeSchedule(ctx, req.(*DescribeScheduleRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_UpdateSchedule_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateScheduleRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).UpdateSchedule(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_UpdateSchedule_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).UpdateSchedule(ctx, req.(*UpdateScheduleRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_PatchSchedule_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(PatchScheduleRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).PatchSchedule(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_PatchSchedule_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).PatchSchedule(ctx, req.(*PatchScheduleRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_ListScheduleMatchingTimes_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListScheduleMatchingTimesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).ListScheduleMatchingTimes(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_ListScheduleMatchingTimes_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).ListScheduleMatchingTimes(ctx, req.(*ListScheduleMatchingTimesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_DeleteSchedule_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteScheduleRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).DeleteSchedule(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_DeleteSchedule_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).DeleteSchedule(ctx, req.(*DeleteScheduleRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_ListSchedules_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListSchedulesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).ListSchedules(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_ListSchedules_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).ListSchedules(ctx, req.(*ListSchedulesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_UpdateWorkerBuildIdCompatibility_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateWorkerBuildIdCompatibilityRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).UpdateWorkerBuildIdCompatibility(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_UpdateWorkerBuildIdCompatibility_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).UpdateWorkerBuildIdCompatibility(ctx, req.(*UpdateWorkerBuildIdCompatibilityRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_GetWorkerBuildIdCompatibility_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetWorkerBuildIdCompatibilityRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).GetWorkerBuildIdCompatibility(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_GetWorkerBuildIdCompatibility_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).GetWorkerBuildIdCompatibility(ctx, req.(*GetWorkerBuildIdCompatibilityRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_UpdateWorkerVersioningRules_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateWorkerVersioningRulesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).UpdateWorkerVersioningRules(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_UpdateWorkerVersioningRules_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).UpdateWorkerVersioningRules(ctx, req.(*UpdateWorkerVersioningRulesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_GetWorkerVersioningRules_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetWorkerVersioningRulesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).GetWorkerVersioningRules(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_GetWorkerVersioningRules_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).GetWorkerVersioningRules(ctx, req.(*GetWorkerVersioningRulesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_GetWorkerTaskReachability_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetWorkerTaskReachabilityRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).GetWorkerTaskReachability(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_GetWorkerTaskReachability_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).GetWorkerTaskReachability(ctx, req.(*GetWorkerTaskReachabilityRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_UpdateWorkflowExecution_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateWorkflowExecutionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).UpdateWorkflowExecution(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_UpdateWorkflowExecution_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).UpdateWorkflowExecution(ctx, req.(*UpdateWorkflowExecutionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_PollWorkflowExecutionUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(PollWorkflowExecutionUpdateRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).PollWorkflowExecutionUpdate(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_PollWorkflowExecutionUpdate_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).PollWorkflowExecutionUpdate(ctx, req.(*PollWorkflowExecutionUpdateRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_StartBatchOperation_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(StartBatchOperationRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).StartBatchOperation(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_StartBatchOperation_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).StartBatchOperation(ctx, req.(*StartBatchOperationRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_StopBatchOperation_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(StopBatchOperationRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).StopBatchOperation(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_StopBatchOperation_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).StopBatchOperation(ctx, req.(*StopBatchOperationRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_DescribeBatchOperation_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DescribeBatchOperationRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).DescribeBatchOperation(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_DescribeBatchOperation_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).DescribeBatchOperation(ctx, req.(*DescribeBatchOperationRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_ListBatchOperations_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListBatchOperationsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).ListBatchOperations(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_ListBatchOperations_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).ListBatchOperations(ctx, req.(*ListBatchOperationsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_PollNexusTaskQueue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(PollNexusTaskQueueRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).PollNexusTaskQueue(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_PollNexusTaskQueue_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).PollNexusTaskQueue(ctx, req.(*PollNexusTaskQueueRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RespondNexusTaskCompleted_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RespondNexusTaskCompletedRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RespondNexusTaskCompleted(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RespondNexusTaskCompleted_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RespondNexusTaskCompleted(ctx, req.(*RespondNexusTaskCompletedRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _WorkflowService_RespondNexusTaskFailed_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RespondNexusTaskFailedRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(WorkflowServiceServer).RespondNexusTaskFailed(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: WorkflowService_RespondNexusTaskFailed_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(WorkflowServiceServer).RespondNexusTaskFailed(ctx, req.(*RespondNexusTaskFailedRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// WorkflowService_ServiceDesc is the grpc.ServiceDesc for WorkflowService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var WorkflowService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "temporal.api.workflowservice.v1.WorkflowService", + HandlerType: (*WorkflowServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "RegisterNamespace", + Handler: _WorkflowService_RegisterNamespace_Handler, + }, + { + MethodName: "DescribeNamespace", + Handler: _WorkflowService_DescribeNamespace_Handler, + }, + { + MethodName: "ListNamespaces", + Handler: _WorkflowService_ListNamespaces_Handler, + }, + { + MethodName: "UpdateNamespace", + Handler: _WorkflowService_UpdateNamespace_Handler, + }, + { + MethodName: "DeprecateNamespace", + Handler: _WorkflowService_DeprecateNamespace_Handler, + }, + { + MethodName: "StartWorkflowExecution", + Handler: _WorkflowService_StartWorkflowExecution_Handler, + }, + { + MethodName: "ExecuteMultiOperation", + Handler: _WorkflowService_ExecuteMultiOperation_Handler, + }, + { + MethodName: "GetWorkflowExecutionHistory", + Handler: _WorkflowService_GetWorkflowExecutionHistory_Handler, + }, + { + MethodName: "GetWorkflowExecutionHistoryReverse", + Handler: _WorkflowService_GetWorkflowExecutionHistoryReverse_Handler, + }, + { + MethodName: "PollWorkflowTaskQueue", + Handler: _WorkflowService_PollWorkflowTaskQueue_Handler, + }, + { + MethodName: "RespondWorkflowTaskCompleted", + Handler: _WorkflowService_RespondWorkflowTaskCompleted_Handler, + }, + { + MethodName: "RespondWorkflowTaskFailed", + Handler: _WorkflowService_RespondWorkflowTaskFailed_Handler, + }, + { + MethodName: "PollActivityTaskQueue", + Handler: _WorkflowService_PollActivityTaskQueue_Handler, + }, + { + MethodName: "RecordActivityTaskHeartbeat", + Handler: _WorkflowService_RecordActivityTaskHeartbeat_Handler, + }, + { + MethodName: "RecordActivityTaskHeartbeatById", + Handler: _WorkflowService_RecordActivityTaskHeartbeatById_Handler, + }, + { + MethodName: "RespondActivityTaskCompleted", + Handler: _WorkflowService_RespondActivityTaskCompleted_Handler, + }, + { + MethodName: "RespondActivityTaskCompletedById", + Handler: _WorkflowService_RespondActivityTaskCompletedById_Handler, + }, + { + MethodName: "RespondActivityTaskFailed", + Handler: _WorkflowService_RespondActivityTaskFailed_Handler, + }, + { + MethodName: "RespondActivityTaskFailedById", + Handler: _WorkflowService_RespondActivityTaskFailedById_Handler, + }, + { + MethodName: "RespondActivityTaskCanceled", + Handler: _WorkflowService_RespondActivityTaskCanceled_Handler, + }, + { + MethodName: "RespondActivityTaskCanceledById", + Handler: _WorkflowService_RespondActivityTaskCanceledById_Handler, + }, + { + MethodName: "RequestCancelWorkflowExecution", + Handler: _WorkflowService_RequestCancelWorkflowExecution_Handler, + }, + { + MethodName: "SignalWorkflowExecution", + Handler: _WorkflowService_SignalWorkflowExecution_Handler, + }, + { + MethodName: "SignalWithStartWorkflowExecution", + Handler: _WorkflowService_SignalWithStartWorkflowExecution_Handler, + }, + { + MethodName: "ResetWorkflowExecution", + Handler: _WorkflowService_ResetWorkflowExecution_Handler, + }, + { + MethodName: "TerminateWorkflowExecution", + Handler: _WorkflowService_TerminateWorkflowExecution_Handler, + }, + { + MethodName: "DeleteWorkflowExecution", + Handler: _WorkflowService_DeleteWorkflowExecution_Handler, + }, + { + MethodName: "ListOpenWorkflowExecutions", + Handler: _WorkflowService_ListOpenWorkflowExecutions_Handler, + }, + { + MethodName: "ListClosedWorkflowExecutions", + Handler: _WorkflowService_ListClosedWorkflowExecutions_Handler, + }, + { + MethodName: "ListWorkflowExecutions", + Handler: _WorkflowService_ListWorkflowExecutions_Handler, + }, + { + MethodName: "ListArchivedWorkflowExecutions", + Handler: _WorkflowService_ListArchivedWorkflowExecutions_Handler, + }, + { + MethodName: "ScanWorkflowExecutions", + Handler: _WorkflowService_ScanWorkflowExecutions_Handler, + }, + { + MethodName: "CountWorkflowExecutions", + Handler: _WorkflowService_CountWorkflowExecutions_Handler, + }, + { + MethodName: "GetSearchAttributes", + Handler: _WorkflowService_GetSearchAttributes_Handler, + }, + { + MethodName: "RespondQueryTaskCompleted", + Handler: _WorkflowService_RespondQueryTaskCompleted_Handler, + }, + { + MethodName: "ResetStickyTaskQueue", + Handler: _WorkflowService_ResetStickyTaskQueue_Handler, + }, + { + MethodName: "QueryWorkflow", + Handler: _WorkflowService_QueryWorkflow_Handler, + }, + { + MethodName: "DescribeWorkflowExecution", + Handler: _WorkflowService_DescribeWorkflowExecution_Handler, + }, + { + MethodName: "DescribeTaskQueue", + Handler: _WorkflowService_DescribeTaskQueue_Handler, + }, + { + MethodName: "GetClusterInfo", + Handler: _WorkflowService_GetClusterInfo_Handler, + }, + { + MethodName: "GetSystemInfo", + Handler: _WorkflowService_GetSystemInfo_Handler, + }, + { + MethodName: "ListTaskQueuePartitions", + Handler: _WorkflowService_ListTaskQueuePartitions_Handler, + }, + { + MethodName: "CreateSchedule", + Handler: _WorkflowService_CreateSchedule_Handler, + }, + { + MethodName: "DescribeSchedule", + Handler: _WorkflowService_DescribeSchedule_Handler, + }, + { + MethodName: "UpdateSchedule", + Handler: _WorkflowService_UpdateSchedule_Handler, + }, + { + MethodName: "PatchSchedule", + Handler: _WorkflowService_PatchSchedule_Handler, + }, + { + MethodName: "ListScheduleMatchingTimes", + Handler: _WorkflowService_ListScheduleMatchingTimes_Handler, + }, + { + MethodName: "DeleteSchedule", + Handler: _WorkflowService_DeleteSchedule_Handler, + }, + { + MethodName: "ListSchedules", + Handler: _WorkflowService_ListSchedules_Handler, + }, + { + MethodName: "UpdateWorkerBuildIdCompatibility", + Handler: _WorkflowService_UpdateWorkerBuildIdCompatibility_Handler, + }, + { + MethodName: "GetWorkerBuildIdCompatibility", + Handler: _WorkflowService_GetWorkerBuildIdCompatibility_Handler, + }, + { + MethodName: "UpdateWorkerVersioningRules", + Handler: _WorkflowService_UpdateWorkerVersioningRules_Handler, + }, + { + MethodName: "GetWorkerVersioningRules", + Handler: _WorkflowService_GetWorkerVersioningRules_Handler, + }, + { + MethodName: "GetWorkerTaskReachability", + Handler: _WorkflowService_GetWorkerTaskReachability_Handler, + }, + { + MethodName: "UpdateWorkflowExecution", + Handler: _WorkflowService_UpdateWorkflowExecution_Handler, + }, + { + MethodName: "PollWorkflowExecutionUpdate", + Handler: _WorkflowService_PollWorkflowExecutionUpdate_Handler, + }, + { + MethodName: "StartBatchOperation", + Handler: _WorkflowService_StartBatchOperation_Handler, + }, + { + MethodName: "StopBatchOperation", + Handler: _WorkflowService_StopBatchOperation_Handler, + }, + { + MethodName: "DescribeBatchOperation", + Handler: _WorkflowService_DescribeBatchOperation_Handler, + }, + { + MethodName: "ListBatchOperations", + Handler: _WorkflowService_ListBatchOperations_Handler, + }, + { + MethodName: "PollNexusTaskQueue", + Handler: _WorkflowService_PollNexusTaskQueue_Handler, + }, + { + MethodName: "RespondNexusTaskCompleted", + Handler: _WorkflowService_RespondNexusTaskCompleted_Handler, + }, + { + MethodName: "RespondNexusTaskFailed", + Handler: _WorkflowService_RespondNexusTaskFailed_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "temporal/api/workflowservice/v1/service.proto", +} diff --git a/vendor/go.temporal.io/api/workflowservicemock/v1/service_grpc.pb.mock.go b/vendor/go.temporal.io/api/workflowservicemock/v1/service_grpc.pb.mock.go new file mode 100644 index 00000000000..a01e90b1cd8 --- /dev/null +++ b/vendor/go.temporal.io/api/workflowservicemock/v1/service_grpc.pb.mock.go @@ -0,0 +1,2334 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by MockGen. DO NOT EDIT. +// Source: workflowservice/v1/service_grpc.pb.go + +// Package workflowservicemock is a generated GoMock package. +package workflowservicemock + +import ( + context "context" + reflect "reflect" + + gomock "github.com/golang/mock/gomock" + workflowservice "go.temporal.io/api/workflowservice/v1" + grpc "google.golang.org/grpc" +) + +// MockWorkflowServiceClient is a mock of WorkflowServiceClient interface. +type MockWorkflowServiceClient struct { + ctrl *gomock.Controller + recorder *MockWorkflowServiceClientMockRecorder +} + +// MockWorkflowServiceClientMockRecorder is the mock recorder for MockWorkflowServiceClient. +type MockWorkflowServiceClientMockRecorder struct { + mock *MockWorkflowServiceClient +} + +// NewMockWorkflowServiceClient creates a new mock instance. +func NewMockWorkflowServiceClient(ctrl *gomock.Controller) *MockWorkflowServiceClient { + mock := &MockWorkflowServiceClient{ctrl: ctrl} + mock.recorder = &MockWorkflowServiceClientMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockWorkflowServiceClient) EXPECT() *MockWorkflowServiceClientMockRecorder { + return m.recorder +} + +// CountWorkflowExecutions mocks base method. +func (m *MockWorkflowServiceClient) CountWorkflowExecutions(ctx context.Context, in *workflowservice.CountWorkflowExecutionsRequest, opts ...grpc.CallOption) (*workflowservice.CountWorkflowExecutionsResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "CountWorkflowExecutions", varargs...) + ret0, _ := ret[0].(*workflowservice.CountWorkflowExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// CountWorkflowExecutions indicates an expected call of CountWorkflowExecutions. +func (mr *MockWorkflowServiceClientMockRecorder) CountWorkflowExecutions(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CountWorkflowExecutions", reflect.TypeOf((*MockWorkflowServiceClient)(nil).CountWorkflowExecutions), varargs...) +} + +// CreateSchedule mocks base method. +func (m *MockWorkflowServiceClient) CreateSchedule(ctx context.Context, in *workflowservice.CreateScheduleRequest, opts ...grpc.CallOption) (*workflowservice.CreateScheduleResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "CreateSchedule", varargs...) + ret0, _ := ret[0].(*workflowservice.CreateScheduleResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// CreateSchedule indicates an expected call of CreateSchedule. +func (mr *MockWorkflowServiceClientMockRecorder) CreateSchedule(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateSchedule", reflect.TypeOf((*MockWorkflowServiceClient)(nil).CreateSchedule), varargs...) +} + +// DeleteSchedule mocks base method. +func (m *MockWorkflowServiceClient) DeleteSchedule(ctx context.Context, in *workflowservice.DeleteScheduleRequest, opts ...grpc.CallOption) (*workflowservice.DeleteScheduleResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "DeleteSchedule", varargs...) + ret0, _ := ret[0].(*workflowservice.DeleteScheduleResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DeleteSchedule indicates an expected call of DeleteSchedule. +func (mr *MockWorkflowServiceClientMockRecorder) DeleteSchedule(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteSchedule", reflect.TypeOf((*MockWorkflowServiceClient)(nil).DeleteSchedule), varargs...) +} + +// DeleteWorkflowExecution mocks base method. +func (m *MockWorkflowServiceClient) DeleteWorkflowExecution(ctx context.Context, in *workflowservice.DeleteWorkflowExecutionRequest, opts ...grpc.CallOption) (*workflowservice.DeleteWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "DeleteWorkflowExecution", varargs...) + ret0, _ := ret[0].(*workflowservice.DeleteWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DeleteWorkflowExecution indicates an expected call of DeleteWorkflowExecution. +func (mr *MockWorkflowServiceClientMockRecorder) DeleteWorkflowExecution(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceClient)(nil).DeleteWorkflowExecution), varargs...) +} + +// DeprecateNamespace mocks base method. +func (m *MockWorkflowServiceClient) DeprecateNamespace(ctx context.Context, in *workflowservice.DeprecateNamespaceRequest, opts ...grpc.CallOption) (*workflowservice.DeprecateNamespaceResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "DeprecateNamespace", varargs...) + ret0, _ := ret[0].(*workflowservice.DeprecateNamespaceResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DeprecateNamespace indicates an expected call of DeprecateNamespace. +func (mr *MockWorkflowServiceClientMockRecorder) DeprecateNamespace(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeprecateNamespace", reflect.TypeOf((*MockWorkflowServiceClient)(nil).DeprecateNamespace), varargs...) +} + +// DescribeBatchOperation mocks base method. +func (m *MockWorkflowServiceClient) DescribeBatchOperation(ctx context.Context, in *workflowservice.DescribeBatchOperationRequest, opts ...grpc.CallOption) (*workflowservice.DescribeBatchOperationResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "DescribeBatchOperation", varargs...) + ret0, _ := ret[0].(*workflowservice.DescribeBatchOperationResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DescribeBatchOperation indicates an expected call of DescribeBatchOperation. +func (mr *MockWorkflowServiceClientMockRecorder) DescribeBatchOperation(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DescribeBatchOperation", reflect.TypeOf((*MockWorkflowServiceClient)(nil).DescribeBatchOperation), varargs...) +} + +// DescribeNamespace mocks base method. +func (m *MockWorkflowServiceClient) DescribeNamespace(ctx context.Context, in *workflowservice.DescribeNamespaceRequest, opts ...grpc.CallOption) (*workflowservice.DescribeNamespaceResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "DescribeNamespace", varargs...) + ret0, _ := ret[0].(*workflowservice.DescribeNamespaceResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DescribeNamespace indicates an expected call of DescribeNamespace. +func (mr *MockWorkflowServiceClientMockRecorder) DescribeNamespace(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DescribeNamespace", reflect.TypeOf((*MockWorkflowServiceClient)(nil).DescribeNamespace), varargs...) +} + +// DescribeSchedule mocks base method. +func (m *MockWorkflowServiceClient) DescribeSchedule(ctx context.Context, in *workflowservice.DescribeScheduleRequest, opts ...grpc.CallOption) (*workflowservice.DescribeScheduleResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "DescribeSchedule", varargs...) + ret0, _ := ret[0].(*workflowservice.DescribeScheduleResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DescribeSchedule indicates an expected call of DescribeSchedule. +func (mr *MockWorkflowServiceClientMockRecorder) DescribeSchedule(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DescribeSchedule", reflect.TypeOf((*MockWorkflowServiceClient)(nil).DescribeSchedule), varargs...) +} + +// DescribeTaskQueue mocks base method. +func (m *MockWorkflowServiceClient) DescribeTaskQueue(ctx context.Context, in *workflowservice.DescribeTaskQueueRequest, opts ...grpc.CallOption) (*workflowservice.DescribeTaskQueueResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "DescribeTaskQueue", varargs...) + ret0, _ := ret[0].(*workflowservice.DescribeTaskQueueResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DescribeTaskQueue indicates an expected call of DescribeTaskQueue. +func (mr *MockWorkflowServiceClientMockRecorder) DescribeTaskQueue(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DescribeTaskQueue", reflect.TypeOf((*MockWorkflowServiceClient)(nil).DescribeTaskQueue), varargs...) +} + +// DescribeWorkflowExecution mocks base method. +func (m *MockWorkflowServiceClient) DescribeWorkflowExecution(ctx context.Context, in *workflowservice.DescribeWorkflowExecutionRequest, opts ...grpc.CallOption) (*workflowservice.DescribeWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "DescribeWorkflowExecution", varargs...) + ret0, _ := ret[0].(*workflowservice.DescribeWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DescribeWorkflowExecution indicates an expected call of DescribeWorkflowExecution. +func (mr *MockWorkflowServiceClientMockRecorder) DescribeWorkflowExecution(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DescribeWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceClient)(nil).DescribeWorkflowExecution), varargs...) +} + +// ExecuteMultiOperation mocks base method. +func (m *MockWorkflowServiceClient) ExecuteMultiOperation(ctx context.Context, in *workflowservice.ExecuteMultiOperationRequest, opts ...grpc.CallOption) (*workflowservice.ExecuteMultiOperationResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "ExecuteMultiOperation", varargs...) + ret0, _ := ret[0].(*workflowservice.ExecuteMultiOperationResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ExecuteMultiOperation indicates an expected call of ExecuteMultiOperation. +func (mr *MockWorkflowServiceClientMockRecorder) ExecuteMultiOperation(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ExecuteMultiOperation", reflect.TypeOf((*MockWorkflowServiceClient)(nil).ExecuteMultiOperation), varargs...) +} + +// GetClusterInfo mocks base method. +func (m *MockWorkflowServiceClient) GetClusterInfo(ctx context.Context, in *workflowservice.GetClusterInfoRequest, opts ...grpc.CallOption) (*workflowservice.GetClusterInfoResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "GetClusterInfo", varargs...) + ret0, _ := ret[0].(*workflowservice.GetClusterInfoResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetClusterInfo indicates an expected call of GetClusterInfo. +func (mr *MockWorkflowServiceClientMockRecorder) GetClusterInfo(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetClusterInfo", reflect.TypeOf((*MockWorkflowServiceClient)(nil).GetClusterInfo), varargs...) +} + +// GetSearchAttributes mocks base method. +func (m *MockWorkflowServiceClient) GetSearchAttributes(ctx context.Context, in *workflowservice.GetSearchAttributesRequest, opts ...grpc.CallOption) (*workflowservice.GetSearchAttributesResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "GetSearchAttributes", varargs...) + ret0, _ := ret[0].(*workflowservice.GetSearchAttributesResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetSearchAttributes indicates an expected call of GetSearchAttributes. +func (mr *MockWorkflowServiceClientMockRecorder) GetSearchAttributes(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSearchAttributes", reflect.TypeOf((*MockWorkflowServiceClient)(nil).GetSearchAttributes), varargs...) +} + +// GetSystemInfo mocks base method. +func (m *MockWorkflowServiceClient) GetSystemInfo(ctx context.Context, in *workflowservice.GetSystemInfoRequest, opts ...grpc.CallOption) (*workflowservice.GetSystemInfoResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "GetSystemInfo", varargs...) + ret0, _ := ret[0].(*workflowservice.GetSystemInfoResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetSystemInfo indicates an expected call of GetSystemInfo. +func (mr *MockWorkflowServiceClientMockRecorder) GetSystemInfo(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSystemInfo", reflect.TypeOf((*MockWorkflowServiceClient)(nil).GetSystemInfo), varargs...) +} + +// GetWorkerBuildIdCompatibility mocks base method. +func (m *MockWorkflowServiceClient) GetWorkerBuildIdCompatibility(ctx context.Context, in *workflowservice.GetWorkerBuildIdCompatibilityRequest, opts ...grpc.CallOption) (*workflowservice.GetWorkerBuildIdCompatibilityResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "GetWorkerBuildIdCompatibility", varargs...) + ret0, _ := ret[0].(*workflowservice.GetWorkerBuildIdCompatibilityResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetWorkerBuildIdCompatibility indicates an expected call of GetWorkerBuildIdCompatibility. +func (mr *MockWorkflowServiceClientMockRecorder) GetWorkerBuildIdCompatibility(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWorkerBuildIdCompatibility", reflect.TypeOf((*MockWorkflowServiceClient)(nil).GetWorkerBuildIdCompatibility), varargs...) +} + +// GetWorkerTaskReachability mocks base method. +func (m *MockWorkflowServiceClient) GetWorkerTaskReachability(ctx context.Context, in *workflowservice.GetWorkerTaskReachabilityRequest, opts ...grpc.CallOption) (*workflowservice.GetWorkerTaskReachabilityResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "GetWorkerTaskReachability", varargs...) + ret0, _ := ret[0].(*workflowservice.GetWorkerTaskReachabilityResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetWorkerTaskReachability indicates an expected call of GetWorkerTaskReachability. +func (mr *MockWorkflowServiceClientMockRecorder) GetWorkerTaskReachability(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWorkerTaskReachability", reflect.TypeOf((*MockWorkflowServiceClient)(nil).GetWorkerTaskReachability), varargs...) +} + +// GetWorkerVersioningRules mocks base method. +func (m *MockWorkflowServiceClient) GetWorkerVersioningRules(ctx context.Context, in *workflowservice.GetWorkerVersioningRulesRequest, opts ...grpc.CallOption) (*workflowservice.GetWorkerVersioningRulesResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "GetWorkerVersioningRules", varargs...) + ret0, _ := ret[0].(*workflowservice.GetWorkerVersioningRulesResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetWorkerVersioningRules indicates an expected call of GetWorkerVersioningRules. +func (mr *MockWorkflowServiceClientMockRecorder) GetWorkerVersioningRules(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWorkerVersioningRules", reflect.TypeOf((*MockWorkflowServiceClient)(nil).GetWorkerVersioningRules), varargs...) +} + +// GetWorkflowExecutionHistory mocks base method. +func (m *MockWorkflowServiceClient) GetWorkflowExecutionHistory(ctx context.Context, in *workflowservice.GetWorkflowExecutionHistoryRequest, opts ...grpc.CallOption) (*workflowservice.GetWorkflowExecutionHistoryResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "GetWorkflowExecutionHistory", varargs...) + ret0, _ := ret[0].(*workflowservice.GetWorkflowExecutionHistoryResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetWorkflowExecutionHistory indicates an expected call of GetWorkflowExecutionHistory. +func (mr *MockWorkflowServiceClientMockRecorder) GetWorkflowExecutionHistory(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWorkflowExecutionHistory", reflect.TypeOf((*MockWorkflowServiceClient)(nil).GetWorkflowExecutionHistory), varargs...) +} + +// GetWorkflowExecutionHistoryReverse mocks base method. +func (m *MockWorkflowServiceClient) GetWorkflowExecutionHistoryReverse(ctx context.Context, in *workflowservice.GetWorkflowExecutionHistoryReverseRequest, opts ...grpc.CallOption) (*workflowservice.GetWorkflowExecutionHistoryReverseResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "GetWorkflowExecutionHistoryReverse", varargs...) + ret0, _ := ret[0].(*workflowservice.GetWorkflowExecutionHistoryReverseResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetWorkflowExecutionHistoryReverse indicates an expected call of GetWorkflowExecutionHistoryReverse. +func (mr *MockWorkflowServiceClientMockRecorder) GetWorkflowExecutionHistoryReverse(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWorkflowExecutionHistoryReverse", reflect.TypeOf((*MockWorkflowServiceClient)(nil).GetWorkflowExecutionHistoryReverse), varargs...) +} + +// ListArchivedWorkflowExecutions mocks base method. +func (m *MockWorkflowServiceClient) ListArchivedWorkflowExecutions(ctx context.Context, in *workflowservice.ListArchivedWorkflowExecutionsRequest, opts ...grpc.CallOption) (*workflowservice.ListArchivedWorkflowExecutionsResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "ListArchivedWorkflowExecutions", varargs...) + ret0, _ := ret[0].(*workflowservice.ListArchivedWorkflowExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListArchivedWorkflowExecutions indicates an expected call of ListArchivedWorkflowExecutions. +func (mr *MockWorkflowServiceClientMockRecorder) ListArchivedWorkflowExecutions(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListArchivedWorkflowExecutions", reflect.TypeOf((*MockWorkflowServiceClient)(nil).ListArchivedWorkflowExecutions), varargs...) +} + +// ListBatchOperations mocks base method. +func (m *MockWorkflowServiceClient) ListBatchOperations(ctx context.Context, in *workflowservice.ListBatchOperationsRequest, opts ...grpc.CallOption) (*workflowservice.ListBatchOperationsResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "ListBatchOperations", varargs...) + ret0, _ := ret[0].(*workflowservice.ListBatchOperationsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListBatchOperations indicates an expected call of ListBatchOperations. +func (mr *MockWorkflowServiceClientMockRecorder) ListBatchOperations(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListBatchOperations", reflect.TypeOf((*MockWorkflowServiceClient)(nil).ListBatchOperations), varargs...) +} + +// ListClosedWorkflowExecutions mocks base method. +func (m *MockWorkflowServiceClient) ListClosedWorkflowExecutions(ctx context.Context, in *workflowservice.ListClosedWorkflowExecutionsRequest, opts ...grpc.CallOption) (*workflowservice.ListClosedWorkflowExecutionsResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "ListClosedWorkflowExecutions", varargs...) + ret0, _ := ret[0].(*workflowservice.ListClosedWorkflowExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListClosedWorkflowExecutions indicates an expected call of ListClosedWorkflowExecutions. +func (mr *MockWorkflowServiceClientMockRecorder) ListClosedWorkflowExecutions(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListClosedWorkflowExecutions", reflect.TypeOf((*MockWorkflowServiceClient)(nil).ListClosedWorkflowExecutions), varargs...) +} + +// ListNamespaces mocks base method. +func (m *MockWorkflowServiceClient) ListNamespaces(ctx context.Context, in *workflowservice.ListNamespacesRequest, opts ...grpc.CallOption) (*workflowservice.ListNamespacesResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "ListNamespaces", varargs...) + ret0, _ := ret[0].(*workflowservice.ListNamespacesResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListNamespaces indicates an expected call of ListNamespaces. +func (mr *MockWorkflowServiceClientMockRecorder) ListNamespaces(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListNamespaces", reflect.TypeOf((*MockWorkflowServiceClient)(nil).ListNamespaces), varargs...) +} + +// ListOpenWorkflowExecutions mocks base method. +func (m *MockWorkflowServiceClient) ListOpenWorkflowExecutions(ctx context.Context, in *workflowservice.ListOpenWorkflowExecutionsRequest, opts ...grpc.CallOption) (*workflowservice.ListOpenWorkflowExecutionsResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "ListOpenWorkflowExecutions", varargs...) + ret0, _ := ret[0].(*workflowservice.ListOpenWorkflowExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListOpenWorkflowExecutions indicates an expected call of ListOpenWorkflowExecutions. +func (mr *MockWorkflowServiceClientMockRecorder) ListOpenWorkflowExecutions(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListOpenWorkflowExecutions", reflect.TypeOf((*MockWorkflowServiceClient)(nil).ListOpenWorkflowExecutions), varargs...) +} + +// ListScheduleMatchingTimes mocks base method. +func (m *MockWorkflowServiceClient) ListScheduleMatchingTimes(ctx context.Context, in *workflowservice.ListScheduleMatchingTimesRequest, opts ...grpc.CallOption) (*workflowservice.ListScheduleMatchingTimesResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "ListScheduleMatchingTimes", varargs...) + ret0, _ := ret[0].(*workflowservice.ListScheduleMatchingTimesResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListScheduleMatchingTimes indicates an expected call of ListScheduleMatchingTimes. +func (mr *MockWorkflowServiceClientMockRecorder) ListScheduleMatchingTimes(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListScheduleMatchingTimes", reflect.TypeOf((*MockWorkflowServiceClient)(nil).ListScheduleMatchingTimes), varargs...) +} + +// ListSchedules mocks base method. +func (m *MockWorkflowServiceClient) ListSchedules(ctx context.Context, in *workflowservice.ListSchedulesRequest, opts ...grpc.CallOption) (*workflowservice.ListSchedulesResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "ListSchedules", varargs...) + ret0, _ := ret[0].(*workflowservice.ListSchedulesResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListSchedules indicates an expected call of ListSchedules. +func (mr *MockWorkflowServiceClientMockRecorder) ListSchedules(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListSchedules", reflect.TypeOf((*MockWorkflowServiceClient)(nil).ListSchedules), varargs...) +} + +// ListTaskQueuePartitions mocks base method. +func (m *MockWorkflowServiceClient) ListTaskQueuePartitions(ctx context.Context, in *workflowservice.ListTaskQueuePartitionsRequest, opts ...grpc.CallOption) (*workflowservice.ListTaskQueuePartitionsResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "ListTaskQueuePartitions", varargs...) + ret0, _ := ret[0].(*workflowservice.ListTaskQueuePartitionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListTaskQueuePartitions indicates an expected call of ListTaskQueuePartitions. +func (mr *MockWorkflowServiceClientMockRecorder) ListTaskQueuePartitions(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListTaskQueuePartitions", reflect.TypeOf((*MockWorkflowServiceClient)(nil).ListTaskQueuePartitions), varargs...) +} + +// ListWorkflowExecutions mocks base method. +func (m *MockWorkflowServiceClient) ListWorkflowExecutions(ctx context.Context, in *workflowservice.ListWorkflowExecutionsRequest, opts ...grpc.CallOption) (*workflowservice.ListWorkflowExecutionsResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "ListWorkflowExecutions", varargs...) + ret0, _ := ret[0].(*workflowservice.ListWorkflowExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListWorkflowExecutions indicates an expected call of ListWorkflowExecutions. +func (mr *MockWorkflowServiceClientMockRecorder) ListWorkflowExecutions(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListWorkflowExecutions", reflect.TypeOf((*MockWorkflowServiceClient)(nil).ListWorkflowExecutions), varargs...) +} + +// PatchSchedule mocks base method. +func (m *MockWorkflowServiceClient) PatchSchedule(ctx context.Context, in *workflowservice.PatchScheduleRequest, opts ...grpc.CallOption) (*workflowservice.PatchScheduleResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "PatchSchedule", varargs...) + ret0, _ := ret[0].(*workflowservice.PatchScheduleResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// PatchSchedule indicates an expected call of PatchSchedule. +func (mr *MockWorkflowServiceClientMockRecorder) PatchSchedule(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PatchSchedule", reflect.TypeOf((*MockWorkflowServiceClient)(nil).PatchSchedule), varargs...) +} + +// PollActivityTaskQueue mocks base method. +func (m *MockWorkflowServiceClient) PollActivityTaskQueue(ctx context.Context, in *workflowservice.PollActivityTaskQueueRequest, opts ...grpc.CallOption) (*workflowservice.PollActivityTaskQueueResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "PollActivityTaskQueue", varargs...) + ret0, _ := ret[0].(*workflowservice.PollActivityTaskQueueResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// PollActivityTaskQueue indicates an expected call of PollActivityTaskQueue. +func (mr *MockWorkflowServiceClientMockRecorder) PollActivityTaskQueue(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PollActivityTaskQueue", reflect.TypeOf((*MockWorkflowServiceClient)(nil).PollActivityTaskQueue), varargs...) +} + +// PollNexusTaskQueue mocks base method. +func (m *MockWorkflowServiceClient) PollNexusTaskQueue(ctx context.Context, in *workflowservice.PollNexusTaskQueueRequest, opts ...grpc.CallOption) (*workflowservice.PollNexusTaskQueueResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "PollNexusTaskQueue", varargs...) + ret0, _ := ret[0].(*workflowservice.PollNexusTaskQueueResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// PollNexusTaskQueue indicates an expected call of PollNexusTaskQueue. +func (mr *MockWorkflowServiceClientMockRecorder) PollNexusTaskQueue(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PollNexusTaskQueue", reflect.TypeOf((*MockWorkflowServiceClient)(nil).PollNexusTaskQueue), varargs...) +} + +// PollWorkflowExecutionUpdate mocks base method. +func (m *MockWorkflowServiceClient) PollWorkflowExecutionUpdate(ctx context.Context, in *workflowservice.PollWorkflowExecutionUpdateRequest, opts ...grpc.CallOption) (*workflowservice.PollWorkflowExecutionUpdateResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "PollWorkflowExecutionUpdate", varargs...) + ret0, _ := ret[0].(*workflowservice.PollWorkflowExecutionUpdateResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// PollWorkflowExecutionUpdate indicates an expected call of PollWorkflowExecutionUpdate. +func (mr *MockWorkflowServiceClientMockRecorder) PollWorkflowExecutionUpdate(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PollWorkflowExecutionUpdate", reflect.TypeOf((*MockWorkflowServiceClient)(nil).PollWorkflowExecutionUpdate), varargs...) +} + +// PollWorkflowTaskQueue mocks base method. +func (m *MockWorkflowServiceClient) PollWorkflowTaskQueue(ctx context.Context, in *workflowservice.PollWorkflowTaskQueueRequest, opts ...grpc.CallOption) (*workflowservice.PollWorkflowTaskQueueResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "PollWorkflowTaskQueue", varargs...) + ret0, _ := ret[0].(*workflowservice.PollWorkflowTaskQueueResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// PollWorkflowTaskQueue indicates an expected call of PollWorkflowTaskQueue. +func (mr *MockWorkflowServiceClientMockRecorder) PollWorkflowTaskQueue(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PollWorkflowTaskQueue", reflect.TypeOf((*MockWorkflowServiceClient)(nil).PollWorkflowTaskQueue), varargs...) +} + +// QueryWorkflow mocks base method. +func (m *MockWorkflowServiceClient) QueryWorkflow(ctx context.Context, in *workflowservice.QueryWorkflowRequest, opts ...grpc.CallOption) (*workflowservice.QueryWorkflowResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "QueryWorkflow", varargs...) + ret0, _ := ret[0].(*workflowservice.QueryWorkflowResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// QueryWorkflow indicates an expected call of QueryWorkflow. +func (mr *MockWorkflowServiceClientMockRecorder) QueryWorkflow(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "QueryWorkflow", reflect.TypeOf((*MockWorkflowServiceClient)(nil).QueryWorkflow), varargs...) +} + +// RecordActivityTaskHeartbeat mocks base method. +func (m *MockWorkflowServiceClient) RecordActivityTaskHeartbeat(ctx context.Context, in *workflowservice.RecordActivityTaskHeartbeatRequest, opts ...grpc.CallOption) (*workflowservice.RecordActivityTaskHeartbeatResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RecordActivityTaskHeartbeat", varargs...) + ret0, _ := ret[0].(*workflowservice.RecordActivityTaskHeartbeatResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RecordActivityTaskHeartbeat indicates an expected call of RecordActivityTaskHeartbeat. +func (mr *MockWorkflowServiceClientMockRecorder) RecordActivityTaskHeartbeat(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecordActivityTaskHeartbeat", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RecordActivityTaskHeartbeat), varargs...) +} + +// RecordActivityTaskHeartbeatById mocks base method. +func (m *MockWorkflowServiceClient) RecordActivityTaskHeartbeatById(ctx context.Context, in *workflowservice.RecordActivityTaskHeartbeatByIdRequest, opts ...grpc.CallOption) (*workflowservice.RecordActivityTaskHeartbeatByIdResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RecordActivityTaskHeartbeatById", varargs...) + ret0, _ := ret[0].(*workflowservice.RecordActivityTaskHeartbeatByIdResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RecordActivityTaskHeartbeatById indicates an expected call of RecordActivityTaskHeartbeatById. +func (mr *MockWorkflowServiceClientMockRecorder) RecordActivityTaskHeartbeatById(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecordActivityTaskHeartbeatById", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RecordActivityTaskHeartbeatById), varargs...) +} + +// RegisterNamespace mocks base method. +func (m *MockWorkflowServiceClient) RegisterNamespace(ctx context.Context, in *workflowservice.RegisterNamespaceRequest, opts ...grpc.CallOption) (*workflowservice.RegisterNamespaceResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RegisterNamespace", varargs...) + ret0, _ := ret[0].(*workflowservice.RegisterNamespaceResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RegisterNamespace indicates an expected call of RegisterNamespace. +func (mr *MockWorkflowServiceClientMockRecorder) RegisterNamespace(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RegisterNamespace", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RegisterNamespace), varargs...) +} + +// RequestCancelWorkflowExecution mocks base method. +func (m *MockWorkflowServiceClient) RequestCancelWorkflowExecution(ctx context.Context, in *workflowservice.RequestCancelWorkflowExecutionRequest, opts ...grpc.CallOption) (*workflowservice.RequestCancelWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RequestCancelWorkflowExecution", varargs...) + ret0, _ := ret[0].(*workflowservice.RequestCancelWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RequestCancelWorkflowExecution indicates an expected call of RequestCancelWorkflowExecution. +func (mr *MockWorkflowServiceClientMockRecorder) RequestCancelWorkflowExecution(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RequestCancelWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RequestCancelWorkflowExecution), varargs...) +} + +// ResetStickyTaskQueue mocks base method. +func (m *MockWorkflowServiceClient) ResetStickyTaskQueue(ctx context.Context, in *workflowservice.ResetStickyTaskQueueRequest, opts ...grpc.CallOption) (*workflowservice.ResetStickyTaskQueueResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "ResetStickyTaskQueue", varargs...) + ret0, _ := ret[0].(*workflowservice.ResetStickyTaskQueueResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ResetStickyTaskQueue indicates an expected call of ResetStickyTaskQueue. +func (mr *MockWorkflowServiceClientMockRecorder) ResetStickyTaskQueue(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetStickyTaskQueue", reflect.TypeOf((*MockWorkflowServiceClient)(nil).ResetStickyTaskQueue), varargs...) +} + +// ResetWorkflowExecution mocks base method. +func (m *MockWorkflowServiceClient) ResetWorkflowExecution(ctx context.Context, in *workflowservice.ResetWorkflowExecutionRequest, opts ...grpc.CallOption) (*workflowservice.ResetWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "ResetWorkflowExecution", varargs...) + ret0, _ := ret[0].(*workflowservice.ResetWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ResetWorkflowExecution indicates an expected call of ResetWorkflowExecution. +func (mr *MockWorkflowServiceClientMockRecorder) ResetWorkflowExecution(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceClient)(nil).ResetWorkflowExecution), varargs...) +} + +// RespondActivityTaskCanceled mocks base method. +func (m *MockWorkflowServiceClient) RespondActivityTaskCanceled(ctx context.Context, in *workflowservice.RespondActivityTaskCanceledRequest, opts ...grpc.CallOption) (*workflowservice.RespondActivityTaskCanceledResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RespondActivityTaskCanceled", varargs...) + ret0, _ := ret[0].(*workflowservice.RespondActivityTaskCanceledResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondActivityTaskCanceled indicates an expected call of RespondActivityTaskCanceled. +func (mr *MockWorkflowServiceClientMockRecorder) RespondActivityTaskCanceled(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondActivityTaskCanceled", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RespondActivityTaskCanceled), varargs...) +} + +// RespondActivityTaskCanceledById mocks base method. +func (m *MockWorkflowServiceClient) RespondActivityTaskCanceledById(ctx context.Context, in *workflowservice.RespondActivityTaskCanceledByIdRequest, opts ...grpc.CallOption) (*workflowservice.RespondActivityTaskCanceledByIdResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RespondActivityTaskCanceledById", varargs...) + ret0, _ := ret[0].(*workflowservice.RespondActivityTaskCanceledByIdResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondActivityTaskCanceledById indicates an expected call of RespondActivityTaskCanceledById. +func (mr *MockWorkflowServiceClientMockRecorder) RespondActivityTaskCanceledById(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondActivityTaskCanceledById", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RespondActivityTaskCanceledById), varargs...) +} + +// RespondActivityTaskCompleted mocks base method. +func (m *MockWorkflowServiceClient) RespondActivityTaskCompleted(ctx context.Context, in *workflowservice.RespondActivityTaskCompletedRequest, opts ...grpc.CallOption) (*workflowservice.RespondActivityTaskCompletedResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RespondActivityTaskCompleted", varargs...) + ret0, _ := ret[0].(*workflowservice.RespondActivityTaskCompletedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondActivityTaskCompleted indicates an expected call of RespondActivityTaskCompleted. +func (mr *MockWorkflowServiceClientMockRecorder) RespondActivityTaskCompleted(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondActivityTaskCompleted", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RespondActivityTaskCompleted), varargs...) +} + +// RespondActivityTaskCompletedById mocks base method. +func (m *MockWorkflowServiceClient) RespondActivityTaskCompletedById(ctx context.Context, in *workflowservice.RespondActivityTaskCompletedByIdRequest, opts ...grpc.CallOption) (*workflowservice.RespondActivityTaskCompletedByIdResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RespondActivityTaskCompletedById", varargs...) + ret0, _ := ret[0].(*workflowservice.RespondActivityTaskCompletedByIdResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondActivityTaskCompletedById indicates an expected call of RespondActivityTaskCompletedById. +func (mr *MockWorkflowServiceClientMockRecorder) RespondActivityTaskCompletedById(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondActivityTaskCompletedById", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RespondActivityTaskCompletedById), varargs...) +} + +// RespondActivityTaskFailed mocks base method. +func (m *MockWorkflowServiceClient) RespondActivityTaskFailed(ctx context.Context, in *workflowservice.RespondActivityTaskFailedRequest, opts ...grpc.CallOption) (*workflowservice.RespondActivityTaskFailedResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RespondActivityTaskFailed", varargs...) + ret0, _ := ret[0].(*workflowservice.RespondActivityTaskFailedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondActivityTaskFailed indicates an expected call of RespondActivityTaskFailed. +func (mr *MockWorkflowServiceClientMockRecorder) RespondActivityTaskFailed(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondActivityTaskFailed", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RespondActivityTaskFailed), varargs...) +} + +// RespondActivityTaskFailedById mocks base method. +func (m *MockWorkflowServiceClient) RespondActivityTaskFailedById(ctx context.Context, in *workflowservice.RespondActivityTaskFailedByIdRequest, opts ...grpc.CallOption) (*workflowservice.RespondActivityTaskFailedByIdResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RespondActivityTaskFailedById", varargs...) + ret0, _ := ret[0].(*workflowservice.RespondActivityTaskFailedByIdResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondActivityTaskFailedById indicates an expected call of RespondActivityTaskFailedById. +func (mr *MockWorkflowServiceClientMockRecorder) RespondActivityTaskFailedById(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondActivityTaskFailedById", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RespondActivityTaskFailedById), varargs...) +} + +// RespondNexusTaskCompleted mocks base method. +func (m *MockWorkflowServiceClient) RespondNexusTaskCompleted(ctx context.Context, in *workflowservice.RespondNexusTaskCompletedRequest, opts ...grpc.CallOption) (*workflowservice.RespondNexusTaskCompletedResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RespondNexusTaskCompleted", varargs...) + ret0, _ := ret[0].(*workflowservice.RespondNexusTaskCompletedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondNexusTaskCompleted indicates an expected call of RespondNexusTaskCompleted. +func (mr *MockWorkflowServiceClientMockRecorder) RespondNexusTaskCompleted(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondNexusTaskCompleted", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RespondNexusTaskCompleted), varargs...) +} + +// RespondNexusTaskFailed mocks base method. +func (m *MockWorkflowServiceClient) RespondNexusTaskFailed(ctx context.Context, in *workflowservice.RespondNexusTaskFailedRequest, opts ...grpc.CallOption) (*workflowservice.RespondNexusTaskFailedResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RespondNexusTaskFailed", varargs...) + ret0, _ := ret[0].(*workflowservice.RespondNexusTaskFailedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondNexusTaskFailed indicates an expected call of RespondNexusTaskFailed. +func (mr *MockWorkflowServiceClientMockRecorder) RespondNexusTaskFailed(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondNexusTaskFailed", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RespondNexusTaskFailed), varargs...) +} + +// RespondQueryTaskCompleted mocks base method. +func (m *MockWorkflowServiceClient) RespondQueryTaskCompleted(ctx context.Context, in *workflowservice.RespondQueryTaskCompletedRequest, opts ...grpc.CallOption) (*workflowservice.RespondQueryTaskCompletedResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RespondQueryTaskCompleted", varargs...) + ret0, _ := ret[0].(*workflowservice.RespondQueryTaskCompletedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondQueryTaskCompleted indicates an expected call of RespondQueryTaskCompleted. +func (mr *MockWorkflowServiceClientMockRecorder) RespondQueryTaskCompleted(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondQueryTaskCompleted", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RespondQueryTaskCompleted), varargs...) +} + +// RespondWorkflowTaskCompleted mocks base method. +func (m *MockWorkflowServiceClient) RespondWorkflowTaskCompleted(ctx context.Context, in *workflowservice.RespondWorkflowTaskCompletedRequest, opts ...grpc.CallOption) (*workflowservice.RespondWorkflowTaskCompletedResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RespondWorkflowTaskCompleted", varargs...) + ret0, _ := ret[0].(*workflowservice.RespondWorkflowTaskCompletedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondWorkflowTaskCompleted indicates an expected call of RespondWorkflowTaskCompleted. +func (mr *MockWorkflowServiceClientMockRecorder) RespondWorkflowTaskCompleted(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondWorkflowTaskCompleted", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RespondWorkflowTaskCompleted), varargs...) +} + +// RespondWorkflowTaskFailed mocks base method. +func (m *MockWorkflowServiceClient) RespondWorkflowTaskFailed(ctx context.Context, in *workflowservice.RespondWorkflowTaskFailedRequest, opts ...grpc.CallOption) (*workflowservice.RespondWorkflowTaskFailedResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RespondWorkflowTaskFailed", varargs...) + ret0, _ := ret[0].(*workflowservice.RespondWorkflowTaskFailedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondWorkflowTaskFailed indicates an expected call of RespondWorkflowTaskFailed. +func (mr *MockWorkflowServiceClientMockRecorder) RespondWorkflowTaskFailed(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondWorkflowTaskFailed", reflect.TypeOf((*MockWorkflowServiceClient)(nil).RespondWorkflowTaskFailed), varargs...) +} + +// ScanWorkflowExecutions mocks base method. +func (m *MockWorkflowServiceClient) ScanWorkflowExecutions(ctx context.Context, in *workflowservice.ScanWorkflowExecutionsRequest, opts ...grpc.CallOption) (*workflowservice.ScanWorkflowExecutionsResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "ScanWorkflowExecutions", varargs...) + ret0, _ := ret[0].(*workflowservice.ScanWorkflowExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ScanWorkflowExecutions indicates an expected call of ScanWorkflowExecutions. +func (mr *MockWorkflowServiceClientMockRecorder) ScanWorkflowExecutions(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ScanWorkflowExecutions", reflect.TypeOf((*MockWorkflowServiceClient)(nil).ScanWorkflowExecutions), varargs...) +} + +// SignalWithStartWorkflowExecution mocks base method. +func (m *MockWorkflowServiceClient) SignalWithStartWorkflowExecution(ctx context.Context, in *workflowservice.SignalWithStartWorkflowExecutionRequest, opts ...grpc.CallOption) (*workflowservice.SignalWithStartWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "SignalWithStartWorkflowExecution", varargs...) + ret0, _ := ret[0].(*workflowservice.SignalWithStartWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// SignalWithStartWorkflowExecution indicates an expected call of SignalWithStartWorkflowExecution. +func (mr *MockWorkflowServiceClientMockRecorder) SignalWithStartWorkflowExecution(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SignalWithStartWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceClient)(nil).SignalWithStartWorkflowExecution), varargs...) +} + +// SignalWorkflowExecution mocks base method. +func (m *MockWorkflowServiceClient) SignalWorkflowExecution(ctx context.Context, in *workflowservice.SignalWorkflowExecutionRequest, opts ...grpc.CallOption) (*workflowservice.SignalWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "SignalWorkflowExecution", varargs...) + ret0, _ := ret[0].(*workflowservice.SignalWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// SignalWorkflowExecution indicates an expected call of SignalWorkflowExecution. +func (mr *MockWorkflowServiceClientMockRecorder) SignalWorkflowExecution(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SignalWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceClient)(nil).SignalWorkflowExecution), varargs...) +} + +// StartBatchOperation mocks base method. +func (m *MockWorkflowServiceClient) StartBatchOperation(ctx context.Context, in *workflowservice.StartBatchOperationRequest, opts ...grpc.CallOption) (*workflowservice.StartBatchOperationResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "StartBatchOperation", varargs...) + ret0, _ := ret[0].(*workflowservice.StartBatchOperationResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// StartBatchOperation indicates an expected call of StartBatchOperation. +func (mr *MockWorkflowServiceClientMockRecorder) StartBatchOperation(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StartBatchOperation", reflect.TypeOf((*MockWorkflowServiceClient)(nil).StartBatchOperation), varargs...) +} + +// StartWorkflowExecution mocks base method. +func (m *MockWorkflowServiceClient) StartWorkflowExecution(ctx context.Context, in *workflowservice.StartWorkflowExecutionRequest, opts ...grpc.CallOption) (*workflowservice.StartWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "StartWorkflowExecution", varargs...) + ret0, _ := ret[0].(*workflowservice.StartWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// StartWorkflowExecution indicates an expected call of StartWorkflowExecution. +func (mr *MockWorkflowServiceClientMockRecorder) StartWorkflowExecution(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StartWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceClient)(nil).StartWorkflowExecution), varargs...) +} + +// StopBatchOperation mocks base method. +func (m *MockWorkflowServiceClient) StopBatchOperation(ctx context.Context, in *workflowservice.StopBatchOperationRequest, opts ...grpc.CallOption) (*workflowservice.StopBatchOperationResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "StopBatchOperation", varargs...) + ret0, _ := ret[0].(*workflowservice.StopBatchOperationResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// StopBatchOperation indicates an expected call of StopBatchOperation. +func (mr *MockWorkflowServiceClientMockRecorder) StopBatchOperation(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StopBatchOperation", reflect.TypeOf((*MockWorkflowServiceClient)(nil).StopBatchOperation), varargs...) +} + +// TerminateWorkflowExecution mocks base method. +func (m *MockWorkflowServiceClient) TerminateWorkflowExecution(ctx context.Context, in *workflowservice.TerminateWorkflowExecutionRequest, opts ...grpc.CallOption) (*workflowservice.TerminateWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "TerminateWorkflowExecution", varargs...) + ret0, _ := ret[0].(*workflowservice.TerminateWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// TerminateWorkflowExecution indicates an expected call of TerminateWorkflowExecution. +func (mr *MockWorkflowServiceClientMockRecorder) TerminateWorkflowExecution(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TerminateWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceClient)(nil).TerminateWorkflowExecution), varargs...) +} + +// UpdateNamespace mocks base method. +func (m *MockWorkflowServiceClient) UpdateNamespace(ctx context.Context, in *workflowservice.UpdateNamespaceRequest, opts ...grpc.CallOption) (*workflowservice.UpdateNamespaceResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "UpdateNamespace", varargs...) + ret0, _ := ret[0].(*workflowservice.UpdateNamespaceResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UpdateNamespace indicates an expected call of UpdateNamespace. +func (mr *MockWorkflowServiceClientMockRecorder) UpdateNamespace(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateNamespace", reflect.TypeOf((*MockWorkflowServiceClient)(nil).UpdateNamespace), varargs...) +} + +// UpdateSchedule mocks base method. +func (m *MockWorkflowServiceClient) UpdateSchedule(ctx context.Context, in *workflowservice.UpdateScheduleRequest, opts ...grpc.CallOption) (*workflowservice.UpdateScheduleResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "UpdateSchedule", varargs...) + ret0, _ := ret[0].(*workflowservice.UpdateScheduleResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UpdateSchedule indicates an expected call of UpdateSchedule. +func (mr *MockWorkflowServiceClientMockRecorder) UpdateSchedule(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateSchedule", reflect.TypeOf((*MockWorkflowServiceClient)(nil).UpdateSchedule), varargs...) +} + +// UpdateWorkerBuildIdCompatibility mocks base method. +func (m *MockWorkflowServiceClient) UpdateWorkerBuildIdCompatibility(ctx context.Context, in *workflowservice.UpdateWorkerBuildIdCompatibilityRequest, opts ...grpc.CallOption) (*workflowservice.UpdateWorkerBuildIdCompatibilityResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "UpdateWorkerBuildIdCompatibility", varargs...) + ret0, _ := ret[0].(*workflowservice.UpdateWorkerBuildIdCompatibilityResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UpdateWorkerBuildIdCompatibility indicates an expected call of UpdateWorkerBuildIdCompatibility. +func (mr *MockWorkflowServiceClientMockRecorder) UpdateWorkerBuildIdCompatibility(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateWorkerBuildIdCompatibility", reflect.TypeOf((*MockWorkflowServiceClient)(nil).UpdateWorkerBuildIdCompatibility), varargs...) +} + +// UpdateWorkerVersioningRules mocks base method. +func (m *MockWorkflowServiceClient) UpdateWorkerVersioningRules(ctx context.Context, in *workflowservice.UpdateWorkerVersioningRulesRequest, opts ...grpc.CallOption) (*workflowservice.UpdateWorkerVersioningRulesResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "UpdateWorkerVersioningRules", varargs...) + ret0, _ := ret[0].(*workflowservice.UpdateWorkerVersioningRulesResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UpdateWorkerVersioningRules indicates an expected call of UpdateWorkerVersioningRules. +func (mr *MockWorkflowServiceClientMockRecorder) UpdateWorkerVersioningRules(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateWorkerVersioningRules", reflect.TypeOf((*MockWorkflowServiceClient)(nil).UpdateWorkerVersioningRules), varargs...) +} + +// UpdateWorkflowExecution mocks base method. +func (m *MockWorkflowServiceClient) UpdateWorkflowExecution(ctx context.Context, in *workflowservice.UpdateWorkflowExecutionRequest, opts ...grpc.CallOption) (*workflowservice.UpdateWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "UpdateWorkflowExecution", varargs...) + ret0, _ := ret[0].(*workflowservice.UpdateWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UpdateWorkflowExecution indicates an expected call of UpdateWorkflowExecution. +func (mr *MockWorkflowServiceClientMockRecorder) UpdateWorkflowExecution(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceClient)(nil).UpdateWorkflowExecution), varargs...) +} + +// MockWorkflowServiceServer is a mock of WorkflowServiceServer interface. +type MockWorkflowServiceServer struct { + ctrl *gomock.Controller + recorder *MockWorkflowServiceServerMockRecorder +} + +// MockWorkflowServiceServerMockRecorder is the mock recorder for MockWorkflowServiceServer. +type MockWorkflowServiceServerMockRecorder struct { + mock *MockWorkflowServiceServer +} + +// NewMockWorkflowServiceServer creates a new mock instance. +func NewMockWorkflowServiceServer(ctrl *gomock.Controller) *MockWorkflowServiceServer { + mock := &MockWorkflowServiceServer{ctrl: ctrl} + mock.recorder = &MockWorkflowServiceServerMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockWorkflowServiceServer) EXPECT() *MockWorkflowServiceServerMockRecorder { + return m.recorder +} + +// CountWorkflowExecutions mocks base method. +func (m *MockWorkflowServiceServer) CountWorkflowExecutions(arg0 context.Context, arg1 *workflowservice.CountWorkflowExecutionsRequest) (*workflowservice.CountWorkflowExecutionsResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CountWorkflowExecutions", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.CountWorkflowExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// CountWorkflowExecutions indicates an expected call of CountWorkflowExecutions. +func (mr *MockWorkflowServiceServerMockRecorder) CountWorkflowExecutions(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CountWorkflowExecutions", reflect.TypeOf((*MockWorkflowServiceServer)(nil).CountWorkflowExecutions), arg0, arg1) +} + +// CreateSchedule mocks base method. +func (m *MockWorkflowServiceServer) CreateSchedule(arg0 context.Context, arg1 *workflowservice.CreateScheduleRequest) (*workflowservice.CreateScheduleResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CreateSchedule", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.CreateScheduleResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// CreateSchedule indicates an expected call of CreateSchedule. +func (mr *MockWorkflowServiceServerMockRecorder) CreateSchedule(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateSchedule", reflect.TypeOf((*MockWorkflowServiceServer)(nil).CreateSchedule), arg0, arg1) +} + +// DeleteSchedule mocks base method. +func (m *MockWorkflowServiceServer) DeleteSchedule(arg0 context.Context, arg1 *workflowservice.DeleteScheduleRequest) (*workflowservice.DeleteScheduleResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DeleteSchedule", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.DeleteScheduleResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DeleteSchedule indicates an expected call of DeleteSchedule. +func (mr *MockWorkflowServiceServerMockRecorder) DeleteSchedule(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteSchedule", reflect.TypeOf((*MockWorkflowServiceServer)(nil).DeleteSchedule), arg0, arg1) +} + +// DeleteWorkflowExecution mocks base method. +func (m *MockWorkflowServiceServer) DeleteWorkflowExecution(arg0 context.Context, arg1 *workflowservice.DeleteWorkflowExecutionRequest) (*workflowservice.DeleteWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DeleteWorkflowExecution", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.DeleteWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DeleteWorkflowExecution indicates an expected call of DeleteWorkflowExecution. +func (mr *MockWorkflowServiceServerMockRecorder) DeleteWorkflowExecution(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceServer)(nil).DeleteWorkflowExecution), arg0, arg1) +} + +// DeprecateNamespace mocks base method. +func (m *MockWorkflowServiceServer) DeprecateNamespace(arg0 context.Context, arg1 *workflowservice.DeprecateNamespaceRequest) (*workflowservice.DeprecateNamespaceResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DeprecateNamespace", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.DeprecateNamespaceResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DeprecateNamespace indicates an expected call of DeprecateNamespace. +func (mr *MockWorkflowServiceServerMockRecorder) DeprecateNamespace(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeprecateNamespace", reflect.TypeOf((*MockWorkflowServiceServer)(nil).DeprecateNamespace), arg0, arg1) +} + +// DescribeBatchOperation mocks base method. +func (m *MockWorkflowServiceServer) DescribeBatchOperation(arg0 context.Context, arg1 *workflowservice.DescribeBatchOperationRequest) (*workflowservice.DescribeBatchOperationResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DescribeBatchOperation", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.DescribeBatchOperationResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DescribeBatchOperation indicates an expected call of DescribeBatchOperation. +func (mr *MockWorkflowServiceServerMockRecorder) DescribeBatchOperation(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DescribeBatchOperation", reflect.TypeOf((*MockWorkflowServiceServer)(nil).DescribeBatchOperation), arg0, arg1) +} + +// DescribeNamespace mocks base method. +func (m *MockWorkflowServiceServer) DescribeNamespace(arg0 context.Context, arg1 *workflowservice.DescribeNamespaceRequest) (*workflowservice.DescribeNamespaceResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DescribeNamespace", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.DescribeNamespaceResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DescribeNamespace indicates an expected call of DescribeNamespace. +func (mr *MockWorkflowServiceServerMockRecorder) DescribeNamespace(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DescribeNamespace", reflect.TypeOf((*MockWorkflowServiceServer)(nil).DescribeNamespace), arg0, arg1) +} + +// DescribeSchedule mocks base method. +func (m *MockWorkflowServiceServer) DescribeSchedule(arg0 context.Context, arg1 *workflowservice.DescribeScheduleRequest) (*workflowservice.DescribeScheduleResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DescribeSchedule", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.DescribeScheduleResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DescribeSchedule indicates an expected call of DescribeSchedule. +func (mr *MockWorkflowServiceServerMockRecorder) DescribeSchedule(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DescribeSchedule", reflect.TypeOf((*MockWorkflowServiceServer)(nil).DescribeSchedule), arg0, arg1) +} + +// DescribeTaskQueue mocks base method. +func (m *MockWorkflowServiceServer) DescribeTaskQueue(arg0 context.Context, arg1 *workflowservice.DescribeTaskQueueRequest) (*workflowservice.DescribeTaskQueueResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DescribeTaskQueue", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.DescribeTaskQueueResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DescribeTaskQueue indicates an expected call of DescribeTaskQueue. +func (mr *MockWorkflowServiceServerMockRecorder) DescribeTaskQueue(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DescribeTaskQueue", reflect.TypeOf((*MockWorkflowServiceServer)(nil).DescribeTaskQueue), arg0, arg1) +} + +// DescribeWorkflowExecution mocks base method. +func (m *MockWorkflowServiceServer) DescribeWorkflowExecution(arg0 context.Context, arg1 *workflowservice.DescribeWorkflowExecutionRequest) (*workflowservice.DescribeWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DescribeWorkflowExecution", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.DescribeWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DescribeWorkflowExecution indicates an expected call of DescribeWorkflowExecution. +func (mr *MockWorkflowServiceServerMockRecorder) DescribeWorkflowExecution(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DescribeWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceServer)(nil).DescribeWorkflowExecution), arg0, arg1) +} + +// ExecuteMultiOperation mocks base method. +func (m *MockWorkflowServiceServer) ExecuteMultiOperation(arg0 context.Context, arg1 *workflowservice.ExecuteMultiOperationRequest) (*workflowservice.ExecuteMultiOperationResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ExecuteMultiOperation", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.ExecuteMultiOperationResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ExecuteMultiOperation indicates an expected call of ExecuteMultiOperation. +func (mr *MockWorkflowServiceServerMockRecorder) ExecuteMultiOperation(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ExecuteMultiOperation", reflect.TypeOf((*MockWorkflowServiceServer)(nil).ExecuteMultiOperation), arg0, arg1) +} + +// GetClusterInfo mocks base method. +func (m *MockWorkflowServiceServer) GetClusterInfo(arg0 context.Context, arg1 *workflowservice.GetClusterInfoRequest) (*workflowservice.GetClusterInfoResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetClusterInfo", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.GetClusterInfoResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetClusterInfo indicates an expected call of GetClusterInfo. +func (mr *MockWorkflowServiceServerMockRecorder) GetClusterInfo(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetClusterInfo", reflect.TypeOf((*MockWorkflowServiceServer)(nil).GetClusterInfo), arg0, arg1) +} + +// GetSearchAttributes mocks base method. +func (m *MockWorkflowServiceServer) GetSearchAttributes(arg0 context.Context, arg1 *workflowservice.GetSearchAttributesRequest) (*workflowservice.GetSearchAttributesResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetSearchAttributes", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.GetSearchAttributesResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetSearchAttributes indicates an expected call of GetSearchAttributes. +func (mr *MockWorkflowServiceServerMockRecorder) GetSearchAttributes(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSearchAttributes", reflect.TypeOf((*MockWorkflowServiceServer)(nil).GetSearchAttributes), arg0, arg1) +} + +// GetSystemInfo mocks base method. +func (m *MockWorkflowServiceServer) GetSystemInfo(arg0 context.Context, arg1 *workflowservice.GetSystemInfoRequest) (*workflowservice.GetSystemInfoResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetSystemInfo", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.GetSystemInfoResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetSystemInfo indicates an expected call of GetSystemInfo. +func (mr *MockWorkflowServiceServerMockRecorder) GetSystemInfo(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSystemInfo", reflect.TypeOf((*MockWorkflowServiceServer)(nil).GetSystemInfo), arg0, arg1) +} + +// GetWorkerBuildIdCompatibility mocks base method. +func (m *MockWorkflowServiceServer) GetWorkerBuildIdCompatibility(arg0 context.Context, arg1 *workflowservice.GetWorkerBuildIdCompatibilityRequest) (*workflowservice.GetWorkerBuildIdCompatibilityResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetWorkerBuildIdCompatibility", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.GetWorkerBuildIdCompatibilityResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetWorkerBuildIdCompatibility indicates an expected call of GetWorkerBuildIdCompatibility. +func (mr *MockWorkflowServiceServerMockRecorder) GetWorkerBuildIdCompatibility(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWorkerBuildIdCompatibility", reflect.TypeOf((*MockWorkflowServiceServer)(nil).GetWorkerBuildIdCompatibility), arg0, arg1) +} + +// GetWorkerTaskReachability mocks base method. +func (m *MockWorkflowServiceServer) GetWorkerTaskReachability(arg0 context.Context, arg1 *workflowservice.GetWorkerTaskReachabilityRequest) (*workflowservice.GetWorkerTaskReachabilityResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetWorkerTaskReachability", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.GetWorkerTaskReachabilityResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetWorkerTaskReachability indicates an expected call of GetWorkerTaskReachability. +func (mr *MockWorkflowServiceServerMockRecorder) GetWorkerTaskReachability(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWorkerTaskReachability", reflect.TypeOf((*MockWorkflowServiceServer)(nil).GetWorkerTaskReachability), arg0, arg1) +} + +// GetWorkerVersioningRules mocks base method. +func (m *MockWorkflowServiceServer) GetWorkerVersioningRules(arg0 context.Context, arg1 *workflowservice.GetWorkerVersioningRulesRequest) (*workflowservice.GetWorkerVersioningRulesResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetWorkerVersioningRules", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.GetWorkerVersioningRulesResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetWorkerVersioningRules indicates an expected call of GetWorkerVersioningRules. +func (mr *MockWorkflowServiceServerMockRecorder) GetWorkerVersioningRules(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWorkerVersioningRules", reflect.TypeOf((*MockWorkflowServiceServer)(nil).GetWorkerVersioningRules), arg0, arg1) +} + +// GetWorkflowExecutionHistory mocks base method. +func (m *MockWorkflowServiceServer) GetWorkflowExecutionHistory(arg0 context.Context, arg1 *workflowservice.GetWorkflowExecutionHistoryRequest) (*workflowservice.GetWorkflowExecutionHistoryResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetWorkflowExecutionHistory", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.GetWorkflowExecutionHistoryResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetWorkflowExecutionHistory indicates an expected call of GetWorkflowExecutionHistory. +func (mr *MockWorkflowServiceServerMockRecorder) GetWorkflowExecutionHistory(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWorkflowExecutionHistory", reflect.TypeOf((*MockWorkflowServiceServer)(nil).GetWorkflowExecutionHistory), arg0, arg1) +} + +// GetWorkflowExecutionHistoryReverse mocks base method. +func (m *MockWorkflowServiceServer) GetWorkflowExecutionHistoryReverse(arg0 context.Context, arg1 *workflowservice.GetWorkflowExecutionHistoryReverseRequest) (*workflowservice.GetWorkflowExecutionHistoryReverseResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetWorkflowExecutionHistoryReverse", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.GetWorkflowExecutionHistoryReverseResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetWorkflowExecutionHistoryReverse indicates an expected call of GetWorkflowExecutionHistoryReverse. +func (mr *MockWorkflowServiceServerMockRecorder) GetWorkflowExecutionHistoryReverse(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWorkflowExecutionHistoryReverse", reflect.TypeOf((*MockWorkflowServiceServer)(nil).GetWorkflowExecutionHistoryReverse), arg0, arg1) +} + +// ListArchivedWorkflowExecutions mocks base method. +func (m *MockWorkflowServiceServer) ListArchivedWorkflowExecutions(arg0 context.Context, arg1 *workflowservice.ListArchivedWorkflowExecutionsRequest) (*workflowservice.ListArchivedWorkflowExecutionsResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ListArchivedWorkflowExecutions", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.ListArchivedWorkflowExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListArchivedWorkflowExecutions indicates an expected call of ListArchivedWorkflowExecutions. +func (mr *MockWorkflowServiceServerMockRecorder) ListArchivedWorkflowExecutions(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListArchivedWorkflowExecutions", reflect.TypeOf((*MockWorkflowServiceServer)(nil).ListArchivedWorkflowExecutions), arg0, arg1) +} + +// ListBatchOperations mocks base method. +func (m *MockWorkflowServiceServer) ListBatchOperations(arg0 context.Context, arg1 *workflowservice.ListBatchOperationsRequest) (*workflowservice.ListBatchOperationsResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ListBatchOperations", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.ListBatchOperationsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListBatchOperations indicates an expected call of ListBatchOperations. +func (mr *MockWorkflowServiceServerMockRecorder) ListBatchOperations(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListBatchOperations", reflect.TypeOf((*MockWorkflowServiceServer)(nil).ListBatchOperations), arg0, arg1) +} + +// ListClosedWorkflowExecutions mocks base method. +func (m *MockWorkflowServiceServer) ListClosedWorkflowExecutions(arg0 context.Context, arg1 *workflowservice.ListClosedWorkflowExecutionsRequest) (*workflowservice.ListClosedWorkflowExecutionsResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ListClosedWorkflowExecutions", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.ListClosedWorkflowExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListClosedWorkflowExecutions indicates an expected call of ListClosedWorkflowExecutions. +func (mr *MockWorkflowServiceServerMockRecorder) ListClosedWorkflowExecutions(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListClosedWorkflowExecutions", reflect.TypeOf((*MockWorkflowServiceServer)(nil).ListClosedWorkflowExecutions), arg0, arg1) +} + +// ListNamespaces mocks base method. +func (m *MockWorkflowServiceServer) ListNamespaces(arg0 context.Context, arg1 *workflowservice.ListNamespacesRequest) (*workflowservice.ListNamespacesResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ListNamespaces", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.ListNamespacesResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListNamespaces indicates an expected call of ListNamespaces. +func (mr *MockWorkflowServiceServerMockRecorder) ListNamespaces(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListNamespaces", reflect.TypeOf((*MockWorkflowServiceServer)(nil).ListNamespaces), arg0, arg1) +} + +// ListOpenWorkflowExecutions mocks base method. +func (m *MockWorkflowServiceServer) ListOpenWorkflowExecutions(arg0 context.Context, arg1 *workflowservice.ListOpenWorkflowExecutionsRequest) (*workflowservice.ListOpenWorkflowExecutionsResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ListOpenWorkflowExecutions", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.ListOpenWorkflowExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListOpenWorkflowExecutions indicates an expected call of ListOpenWorkflowExecutions. +func (mr *MockWorkflowServiceServerMockRecorder) ListOpenWorkflowExecutions(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListOpenWorkflowExecutions", reflect.TypeOf((*MockWorkflowServiceServer)(nil).ListOpenWorkflowExecutions), arg0, arg1) +} + +// ListScheduleMatchingTimes mocks base method. +func (m *MockWorkflowServiceServer) ListScheduleMatchingTimes(arg0 context.Context, arg1 *workflowservice.ListScheduleMatchingTimesRequest) (*workflowservice.ListScheduleMatchingTimesResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ListScheduleMatchingTimes", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.ListScheduleMatchingTimesResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListScheduleMatchingTimes indicates an expected call of ListScheduleMatchingTimes. +func (mr *MockWorkflowServiceServerMockRecorder) ListScheduleMatchingTimes(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListScheduleMatchingTimes", reflect.TypeOf((*MockWorkflowServiceServer)(nil).ListScheduleMatchingTimes), arg0, arg1) +} + +// ListSchedules mocks base method. +func (m *MockWorkflowServiceServer) ListSchedules(arg0 context.Context, arg1 *workflowservice.ListSchedulesRequest) (*workflowservice.ListSchedulesResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ListSchedules", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.ListSchedulesResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListSchedules indicates an expected call of ListSchedules. +func (mr *MockWorkflowServiceServerMockRecorder) ListSchedules(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListSchedules", reflect.TypeOf((*MockWorkflowServiceServer)(nil).ListSchedules), arg0, arg1) +} + +// ListTaskQueuePartitions mocks base method. +func (m *MockWorkflowServiceServer) ListTaskQueuePartitions(arg0 context.Context, arg1 *workflowservice.ListTaskQueuePartitionsRequest) (*workflowservice.ListTaskQueuePartitionsResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ListTaskQueuePartitions", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.ListTaskQueuePartitionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListTaskQueuePartitions indicates an expected call of ListTaskQueuePartitions. +func (mr *MockWorkflowServiceServerMockRecorder) ListTaskQueuePartitions(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListTaskQueuePartitions", reflect.TypeOf((*MockWorkflowServiceServer)(nil).ListTaskQueuePartitions), arg0, arg1) +} + +// ListWorkflowExecutions mocks base method. +func (m *MockWorkflowServiceServer) ListWorkflowExecutions(arg0 context.Context, arg1 *workflowservice.ListWorkflowExecutionsRequest) (*workflowservice.ListWorkflowExecutionsResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ListWorkflowExecutions", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.ListWorkflowExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListWorkflowExecutions indicates an expected call of ListWorkflowExecutions. +func (mr *MockWorkflowServiceServerMockRecorder) ListWorkflowExecutions(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListWorkflowExecutions", reflect.TypeOf((*MockWorkflowServiceServer)(nil).ListWorkflowExecutions), arg0, arg1) +} + +// PatchSchedule mocks base method. +func (m *MockWorkflowServiceServer) PatchSchedule(arg0 context.Context, arg1 *workflowservice.PatchScheduleRequest) (*workflowservice.PatchScheduleResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "PatchSchedule", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.PatchScheduleResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// PatchSchedule indicates an expected call of PatchSchedule. +func (mr *MockWorkflowServiceServerMockRecorder) PatchSchedule(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PatchSchedule", reflect.TypeOf((*MockWorkflowServiceServer)(nil).PatchSchedule), arg0, arg1) +} + +// PollActivityTaskQueue mocks base method. +func (m *MockWorkflowServiceServer) PollActivityTaskQueue(arg0 context.Context, arg1 *workflowservice.PollActivityTaskQueueRequest) (*workflowservice.PollActivityTaskQueueResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "PollActivityTaskQueue", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.PollActivityTaskQueueResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// PollActivityTaskQueue indicates an expected call of PollActivityTaskQueue. +func (mr *MockWorkflowServiceServerMockRecorder) PollActivityTaskQueue(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PollActivityTaskQueue", reflect.TypeOf((*MockWorkflowServiceServer)(nil).PollActivityTaskQueue), arg0, arg1) +} + +// PollNexusTaskQueue mocks base method. +func (m *MockWorkflowServiceServer) PollNexusTaskQueue(arg0 context.Context, arg1 *workflowservice.PollNexusTaskQueueRequest) (*workflowservice.PollNexusTaskQueueResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "PollNexusTaskQueue", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.PollNexusTaskQueueResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// PollNexusTaskQueue indicates an expected call of PollNexusTaskQueue. +func (mr *MockWorkflowServiceServerMockRecorder) PollNexusTaskQueue(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PollNexusTaskQueue", reflect.TypeOf((*MockWorkflowServiceServer)(nil).PollNexusTaskQueue), arg0, arg1) +} + +// PollWorkflowExecutionUpdate mocks base method. +func (m *MockWorkflowServiceServer) PollWorkflowExecutionUpdate(arg0 context.Context, arg1 *workflowservice.PollWorkflowExecutionUpdateRequest) (*workflowservice.PollWorkflowExecutionUpdateResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "PollWorkflowExecutionUpdate", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.PollWorkflowExecutionUpdateResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// PollWorkflowExecutionUpdate indicates an expected call of PollWorkflowExecutionUpdate. +func (mr *MockWorkflowServiceServerMockRecorder) PollWorkflowExecutionUpdate(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PollWorkflowExecutionUpdate", reflect.TypeOf((*MockWorkflowServiceServer)(nil).PollWorkflowExecutionUpdate), arg0, arg1) +} + +// PollWorkflowTaskQueue mocks base method. +func (m *MockWorkflowServiceServer) PollWorkflowTaskQueue(arg0 context.Context, arg1 *workflowservice.PollWorkflowTaskQueueRequest) (*workflowservice.PollWorkflowTaskQueueResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "PollWorkflowTaskQueue", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.PollWorkflowTaskQueueResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// PollWorkflowTaskQueue indicates an expected call of PollWorkflowTaskQueue. +func (mr *MockWorkflowServiceServerMockRecorder) PollWorkflowTaskQueue(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PollWorkflowTaskQueue", reflect.TypeOf((*MockWorkflowServiceServer)(nil).PollWorkflowTaskQueue), arg0, arg1) +} + +// QueryWorkflow mocks base method. +func (m *MockWorkflowServiceServer) QueryWorkflow(arg0 context.Context, arg1 *workflowservice.QueryWorkflowRequest) (*workflowservice.QueryWorkflowResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "QueryWorkflow", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.QueryWorkflowResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// QueryWorkflow indicates an expected call of QueryWorkflow. +func (mr *MockWorkflowServiceServerMockRecorder) QueryWorkflow(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "QueryWorkflow", reflect.TypeOf((*MockWorkflowServiceServer)(nil).QueryWorkflow), arg0, arg1) +} + +// RecordActivityTaskHeartbeat mocks base method. +func (m *MockWorkflowServiceServer) RecordActivityTaskHeartbeat(arg0 context.Context, arg1 *workflowservice.RecordActivityTaskHeartbeatRequest) (*workflowservice.RecordActivityTaskHeartbeatResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RecordActivityTaskHeartbeat", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RecordActivityTaskHeartbeatResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RecordActivityTaskHeartbeat indicates an expected call of RecordActivityTaskHeartbeat. +func (mr *MockWorkflowServiceServerMockRecorder) RecordActivityTaskHeartbeat(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecordActivityTaskHeartbeat", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RecordActivityTaskHeartbeat), arg0, arg1) +} + +// RecordActivityTaskHeartbeatById mocks base method. +func (m *MockWorkflowServiceServer) RecordActivityTaskHeartbeatById(arg0 context.Context, arg1 *workflowservice.RecordActivityTaskHeartbeatByIdRequest) (*workflowservice.RecordActivityTaskHeartbeatByIdResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RecordActivityTaskHeartbeatById", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RecordActivityTaskHeartbeatByIdResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RecordActivityTaskHeartbeatById indicates an expected call of RecordActivityTaskHeartbeatById. +func (mr *MockWorkflowServiceServerMockRecorder) RecordActivityTaskHeartbeatById(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecordActivityTaskHeartbeatById", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RecordActivityTaskHeartbeatById), arg0, arg1) +} + +// RegisterNamespace mocks base method. +func (m *MockWorkflowServiceServer) RegisterNamespace(arg0 context.Context, arg1 *workflowservice.RegisterNamespaceRequest) (*workflowservice.RegisterNamespaceResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RegisterNamespace", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RegisterNamespaceResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RegisterNamespace indicates an expected call of RegisterNamespace. +func (mr *MockWorkflowServiceServerMockRecorder) RegisterNamespace(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RegisterNamespace", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RegisterNamespace), arg0, arg1) +} + +// RequestCancelWorkflowExecution mocks base method. +func (m *MockWorkflowServiceServer) RequestCancelWorkflowExecution(arg0 context.Context, arg1 *workflowservice.RequestCancelWorkflowExecutionRequest) (*workflowservice.RequestCancelWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RequestCancelWorkflowExecution", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RequestCancelWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RequestCancelWorkflowExecution indicates an expected call of RequestCancelWorkflowExecution. +func (mr *MockWorkflowServiceServerMockRecorder) RequestCancelWorkflowExecution(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RequestCancelWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RequestCancelWorkflowExecution), arg0, arg1) +} + +// ResetStickyTaskQueue mocks base method. +func (m *MockWorkflowServiceServer) ResetStickyTaskQueue(arg0 context.Context, arg1 *workflowservice.ResetStickyTaskQueueRequest) (*workflowservice.ResetStickyTaskQueueResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ResetStickyTaskQueue", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.ResetStickyTaskQueueResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ResetStickyTaskQueue indicates an expected call of ResetStickyTaskQueue. +func (mr *MockWorkflowServiceServerMockRecorder) ResetStickyTaskQueue(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetStickyTaskQueue", reflect.TypeOf((*MockWorkflowServiceServer)(nil).ResetStickyTaskQueue), arg0, arg1) +} + +// ResetWorkflowExecution mocks base method. +func (m *MockWorkflowServiceServer) ResetWorkflowExecution(arg0 context.Context, arg1 *workflowservice.ResetWorkflowExecutionRequest) (*workflowservice.ResetWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ResetWorkflowExecution", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.ResetWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ResetWorkflowExecution indicates an expected call of ResetWorkflowExecution. +func (mr *MockWorkflowServiceServerMockRecorder) ResetWorkflowExecution(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceServer)(nil).ResetWorkflowExecution), arg0, arg1) +} + +// RespondActivityTaskCanceled mocks base method. +func (m *MockWorkflowServiceServer) RespondActivityTaskCanceled(arg0 context.Context, arg1 *workflowservice.RespondActivityTaskCanceledRequest) (*workflowservice.RespondActivityTaskCanceledResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RespondActivityTaskCanceled", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RespondActivityTaskCanceledResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondActivityTaskCanceled indicates an expected call of RespondActivityTaskCanceled. +func (mr *MockWorkflowServiceServerMockRecorder) RespondActivityTaskCanceled(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondActivityTaskCanceled", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RespondActivityTaskCanceled), arg0, arg1) +} + +// RespondActivityTaskCanceledById mocks base method. +func (m *MockWorkflowServiceServer) RespondActivityTaskCanceledById(arg0 context.Context, arg1 *workflowservice.RespondActivityTaskCanceledByIdRequest) (*workflowservice.RespondActivityTaskCanceledByIdResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RespondActivityTaskCanceledById", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RespondActivityTaskCanceledByIdResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondActivityTaskCanceledById indicates an expected call of RespondActivityTaskCanceledById. +func (mr *MockWorkflowServiceServerMockRecorder) RespondActivityTaskCanceledById(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondActivityTaskCanceledById", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RespondActivityTaskCanceledById), arg0, arg1) +} + +// RespondActivityTaskCompleted mocks base method. +func (m *MockWorkflowServiceServer) RespondActivityTaskCompleted(arg0 context.Context, arg1 *workflowservice.RespondActivityTaskCompletedRequest) (*workflowservice.RespondActivityTaskCompletedResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RespondActivityTaskCompleted", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RespondActivityTaskCompletedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondActivityTaskCompleted indicates an expected call of RespondActivityTaskCompleted. +func (mr *MockWorkflowServiceServerMockRecorder) RespondActivityTaskCompleted(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondActivityTaskCompleted", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RespondActivityTaskCompleted), arg0, arg1) +} + +// RespondActivityTaskCompletedById mocks base method. +func (m *MockWorkflowServiceServer) RespondActivityTaskCompletedById(arg0 context.Context, arg1 *workflowservice.RespondActivityTaskCompletedByIdRequest) (*workflowservice.RespondActivityTaskCompletedByIdResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RespondActivityTaskCompletedById", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RespondActivityTaskCompletedByIdResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondActivityTaskCompletedById indicates an expected call of RespondActivityTaskCompletedById. +func (mr *MockWorkflowServiceServerMockRecorder) RespondActivityTaskCompletedById(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondActivityTaskCompletedById", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RespondActivityTaskCompletedById), arg0, arg1) +} + +// RespondActivityTaskFailed mocks base method. +func (m *MockWorkflowServiceServer) RespondActivityTaskFailed(arg0 context.Context, arg1 *workflowservice.RespondActivityTaskFailedRequest) (*workflowservice.RespondActivityTaskFailedResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RespondActivityTaskFailed", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RespondActivityTaskFailedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondActivityTaskFailed indicates an expected call of RespondActivityTaskFailed. +func (mr *MockWorkflowServiceServerMockRecorder) RespondActivityTaskFailed(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondActivityTaskFailed", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RespondActivityTaskFailed), arg0, arg1) +} + +// RespondActivityTaskFailedById mocks base method. +func (m *MockWorkflowServiceServer) RespondActivityTaskFailedById(arg0 context.Context, arg1 *workflowservice.RespondActivityTaskFailedByIdRequest) (*workflowservice.RespondActivityTaskFailedByIdResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RespondActivityTaskFailedById", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RespondActivityTaskFailedByIdResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondActivityTaskFailedById indicates an expected call of RespondActivityTaskFailedById. +func (mr *MockWorkflowServiceServerMockRecorder) RespondActivityTaskFailedById(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondActivityTaskFailedById", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RespondActivityTaskFailedById), arg0, arg1) +} + +// RespondNexusTaskCompleted mocks base method. +func (m *MockWorkflowServiceServer) RespondNexusTaskCompleted(arg0 context.Context, arg1 *workflowservice.RespondNexusTaskCompletedRequest) (*workflowservice.RespondNexusTaskCompletedResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RespondNexusTaskCompleted", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RespondNexusTaskCompletedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondNexusTaskCompleted indicates an expected call of RespondNexusTaskCompleted. +func (mr *MockWorkflowServiceServerMockRecorder) RespondNexusTaskCompleted(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondNexusTaskCompleted", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RespondNexusTaskCompleted), arg0, arg1) +} + +// RespondNexusTaskFailed mocks base method. +func (m *MockWorkflowServiceServer) RespondNexusTaskFailed(arg0 context.Context, arg1 *workflowservice.RespondNexusTaskFailedRequest) (*workflowservice.RespondNexusTaskFailedResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RespondNexusTaskFailed", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RespondNexusTaskFailedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondNexusTaskFailed indicates an expected call of RespondNexusTaskFailed. +func (mr *MockWorkflowServiceServerMockRecorder) RespondNexusTaskFailed(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondNexusTaskFailed", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RespondNexusTaskFailed), arg0, arg1) +} + +// RespondQueryTaskCompleted mocks base method. +func (m *MockWorkflowServiceServer) RespondQueryTaskCompleted(arg0 context.Context, arg1 *workflowservice.RespondQueryTaskCompletedRequest) (*workflowservice.RespondQueryTaskCompletedResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RespondQueryTaskCompleted", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RespondQueryTaskCompletedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondQueryTaskCompleted indicates an expected call of RespondQueryTaskCompleted. +func (mr *MockWorkflowServiceServerMockRecorder) RespondQueryTaskCompleted(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondQueryTaskCompleted", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RespondQueryTaskCompleted), arg0, arg1) +} + +// RespondWorkflowTaskCompleted mocks base method. +func (m *MockWorkflowServiceServer) RespondWorkflowTaskCompleted(arg0 context.Context, arg1 *workflowservice.RespondWorkflowTaskCompletedRequest) (*workflowservice.RespondWorkflowTaskCompletedResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RespondWorkflowTaskCompleted", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RespondWorkflowTaskCompletedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondWorkflowTaskCompleted indicates an expected call of RespondWorkflowTaskCompleted. +func (mr *MockWorkflowServiceServerMockRecorder) RespondWorkflowTaskCompleted(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondWorkflowTaskCompleted", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RespondWorkflowTaskCompleted), arg0, arg1) +} + +// RespondWorkflowTaskFailed mocks base method. +func (m *MockWorkflowServiceServer) RespondWorkflowTaskFailed(arg0 context.Context, arg1 *workflowservice.RespondWorkflowTaskFailedRequest) (*workflowservice.RespondWorkflowTaskFailedResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RespondWorkflowTaskFailed", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.RespondWorkflowTaskFailedResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RespondWorkflowTaskFailed indicates an expected call of RespondWorkflowTaskFailed. +func (mr *MockWorkflowServiceServerMockRecorder) RespondWorkflowTaskFailed(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RespondWorkflowTaskFailed", reflect.TypeOf((*MockWorkflowServiceServer)(nil).RespondWorkflowTaskFailed), arg0, arg1) +} + +// ScanWorkflowExecutions mocks base method. +func (m *MockWorkflowServiceServer) ScanWorkflowExecutions(arg0 context.Context, arg1 *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ScanWorkflowExecutions", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.ScanWorkflowExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ScanWorkflowExecutions indicates an expected call of ScanWorkflowExecutions. +func (mr *MockWorkflowServiceServerMockRecorder) ScanWorkflowExecutions(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ScanWorkflowExecutions", reflect.TypeOf((*MockWorkflowServiceServer)(nil).ScanWorkflowExecutions), arg0, arg1) +} + +// SignalWithStartWorkflowExecution mocks base method. +func (m *MockWorkflowServiceServer) SignalWithStartWorkflowExecution(arg0 context.Context, arg1 *workflowservice.SignalWithStartWorkflowExecutionRequest) (*workflowservice.SignalWithStartWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SignalWithStartWorkflowExecution", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.SignalWithStartWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// SignalWithStartWorkflowExecution indicates an expected call of SignalWithStartWorkflowExecution. +func (mr *MockWorkflowServiceServerMockRecorder) SignalWithStartWorkflowExecution(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SignalWithStartWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceServer)(nil).SignalWithStartWorkflowExecution), arg0, arg1) +} + +// SignalWorkflowExecution mocks base method. +func (m *MockWorkflowServiceServer) SignalWorkflowExecution(arg0 context.Context, arg1 *workflowservice.SignalWorkflowExecutionRequest) (*workflowservice.SignalWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SignalWorkflowExecution", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.SignalWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// SignalWorkflowExecution indicates an expected call of SignalWorkflowExecution. +func (mr *MockWorkflowServiceServerMockRecorder) SignalWorkflowExecution(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SignalWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceServer)(nil).SignalWorkflowExecution), arg0, arg1) +} + +// StartBatchOperation mocks base method. +func (m *MockWorkflowServiceServer) StartBatchOperation(arg0 context.Context, arg1 *workflowservice.StartBatchOperationRequest) (*workflowservice.StartBatchOperationResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "StartBatchOperation", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.StartBatchOperationResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// StartBatchOperation indicates an expected call of StartBatchOperation. +func (mr *MockWorkflowServiceServerMockRecorder) StartBatchOperation(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StartBatchOperation", reflect.TypeOf((*MockWorkflowServiceServer)(nil).StartBatchOperation), arg0, arg1) +} + +// StartWorkflowExecution mocks base method. +func (m *MockWorkflowServiceServer) StartWorkflowExecution(arg0 context.Context, arg1 *workflowservice.StartWorkflowExecutionRequest) (*workflowservice.StartWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "StartWorkflowExecution", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.StartWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// StartWorkflowExecution indicates an expected call of StartWorkflowExecution. +func (mr *MockWorkflowServiceServerMockRecorder) StartWorkflowExecution(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StartWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceServer)(nil).StartWorkflowExecution), arg0, arg1) +} + +// StopBatchOperation mocks base method. +func (m *MockWorkflowServiceServer) StopBatchOperation(arg0 context.Context, arg1 *workflowservice.StopBatchOperationRequest) (*workflowservice.StopBatchOperationResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "StopBatchOperation", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.StopBatchOperationResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// StopBatchOperation indicates an expected call of StopBatchOperation. +func (mr *MockWorkflowServiceServerMockRecorder) StopBatchOperation(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StopBatchOperation", reflect.TypeOf((*MockWorkflowServiceServer)(nil).StopBatchOperation), arg0, arg1) +} + +// TerminateWorkflowExecution mocks base method. +func (m *MockWorkflowServiceServer) TerminateWorkflowExecution(arg0 context.Context, arg1 *workflowservice.TerminateWorkflowExecutionRequest) (*workflowservice.TerminateWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "TerminateWorkflowExecution", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.TerminateWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// TerminateWorkflowExecution indicates an expected call of TerminateWorkflowExecution. +func (mr *MockWorkflowServiceServerMockRecorder) TerminateWorkflowExecution(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TerminateWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceServer)(nil).TerminateWorkflowExecution), arg0, arg1) +} + +// UpdateNamespace mocks base method. +func (m *MockWorkflowServiceServer) UpdateNamespace(arg0 context.Context, arg1 *workflowservice.UpdateNamespaceRequest) (*workflowservice.UpdateNamespaceResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "UpdateNamespace", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.UpdateNamespaceResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UpdateNamespace indicates an expected call of UpdateNamespace. +func (mr *MockWorkflowServiceServerMockRecorder) UpdateNamespace(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateNamespace", reflect.TypeOf((*MockWorkflowServiceServer)(nil).UpdateNamespace), arg0, arg1) +} + +// UpdateSchedule mocks base method. +func (m *MockWorkflowServiceServer) UpdateSchedule(arg0 context.Context, arg1 *workflowservice.UpdateScheduleRequest) (*workflowservice.UpdateScheduleResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "UpdateSchedule", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.UpdateScheduleResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UpdateSchedule indicates an expected call of UpdateSchedule. +func (mr *MockWorkflowServiceServerMockRecorder) UpdateSchedule(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateSchedule", reflect.TypeOf((*MockWorkflowServiceServer)(nil).UpdateSchedule), arg0, arg1) +} + +// UpdateWorkerBuildIdCompatibility mocks base method. +func (m *MockWorkflowServiceServer) UpdateWorkerBuildIdCompatibility(arg0 context.Context, arg1 *workflowservice.UpdateWorkerBuildIdCompatibilityRequest) (*workflowservice.UpdateWorkerBuildIdCompatibilityResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "UpdateWorkerBuildIdCompatibility", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.UpdateWorkerBuildIdCompatibilityResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UpdateWorkerBuildIdCompatibility indicates an expected call of UpdateWorkerBuildIdCompatibility. +func (mr *MockWorkflowServiceServerMockRecorder) UpdateWorkerBuildIdCompatibility(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateWorkerBuildIdCompatibility", reflect.TypeOf((*MockWorkflowServiceServer)(nil).UpdateWorkerBuildIdCompatibility), arg0, arg1) +} + +// UpdateWorkerVersioningRules mocks base method. +func (m *MockWorkflowServiceServer) UpdateWorkerVersioningRules(arg0 context.Context, arg1 *workflowservice.UpdateWorkerVersioningRulesRequest) (*workflowservice.UpdateWorkerVersioningRulesResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "UpdateWorkerVersioningRules", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.UpdateWorkerVersioningRulesResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UpdateWorkerVersioningRules indicates an expected call of UpdateWorkerVersioningRules. +func (mr *MockWorkflowServiceServerMockRecorder) UpdateWorkerVersioningRules(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateWorkerVersioningRules", reflect.TypeOf((*MockWorkflowServiceServer)(nil).UpdateWorkerVersioningRules), arg0, arg1) +} + +// UpdateWorkflowExecution mocks base method. +func (m *MockWorkflowServiceServer) UpdateWorkflowExecution(arg0 context.Context, arg1 *workflowservice.UpdateWorkflowExecutionRequest) (*workflowservice.UpdateWorkflowExecutionResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "UpdateWorkflowExecution", arg0, arg1) + ret0, _ := ret[0].(*workflowservice.UpdateWorkflowExecutionResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UpdateWorkflowExecution indicates an expected call of UpdateWorkflowExecution. +func (mr *MockWorkflowServiceServerMockRecorder) UpdateWorkflowExecution(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateWorkflowExecution", reflect.TypeOf((*MockWorkflowServiceServer)(nil).UpdateWorkflowExecution), arg0, arg1) +} + +// mustEmbedUnimplementedWorkflowServiceServer mocks base method. +func (m *MockWorkflowServiceServer) mustEmbedUnimplementedWorkflowServiceServer() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "mustEmbedUnimplementedWorkflowServiceServer") +} + +// mustEmbedUnimplementedWorkflowServiceServer indicates an expected call of mustEmbedUnimplementedWorkflowServiceServer. +func (mr *MockWorkflowServiceServerMockRecorder) mustEmbedUnimplementedWorkflowServiceServer() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "mustEmbedUnimplementedWorkflowServiceServer", reflect.TypeOf((*MockWorkflowServiceServer)(nil).mustEmbedUnimplementedWorkflowServiceServer)) +} + +// MockUnsafeWorkflowServiceServer is a mock of UnsafeWorkflowServiceServer interface. +type MockUnsafeWorkflowServiceServer struct { + ctrl *gomock.Controller + recorder *MockUnsafeWorkflowServiceServerMockRecorder +} + +// MockUnsafeWorkflowServiceServerMockRecorder is the mock recorder for MockUnsafeWorkflowServiceServer. +type MockUnsafeWorkflowServiceServerMockRecorder struct { + mock *MockUnsafeWorkflowServiceServer +} + +// NewMockUnsafeWorkflowServiceServer creates a new mock instance. +func NewMockUnsafeWorkflowServiceServer(ctrl *gomock.Controller) *MockUnsafeWorkflowServiceServer { + mock := &MockUnsafeWorkflowServiceServer{ctrl: ctrl} + mock.recorder = &MockUnsafeWorkflowServiceServerMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockUnsafeWorkflowServiceServer) EXPECT() *MockUnsafeWorkflowServiceServerMockRecorder { + return m.recorder +} + +// mustEmbedUnimplementedWorkflowServiceServer mocks base method. +func (m *MockUnsafeWorkflowServiceServer) mustEmbedUnimplementedWorkflowServiceServer() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "mustEmbedUnimplementedWorkflowServiceServer") +} + +// mustEmbedUnimplementedWorkflowServiceServer indicates an expected call of mustEmbedUnimplementedWorkflowServiceServer. +func (mr *MockUnsafeWorkflowServiceServerMockRecorder) mustEmbedUnimplementedWorkflowServiceServer() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "mustEmbedUnimplementedWorkflowServiceServer", reflect.TypeOf((*MockUnsafeWorkflowServiceServer)(nil).mustEmbedUnimplementedWorkflowServiceServer)) +} diff --git a/vendor/go.temporal.io/sdk/LICENSE b/vendor/go.temporal.io/sdk/LICENSE new file mode 100644 index 00000000000..20a609ec802 --- /dev/null +++ b/vendor/go.temporal.io/sdk/LICENSE @@ -0,0 +1,23 @@ +The MIT License + +Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. + +Copyright (c) 2020 Uber Technologies, Inc. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. \ No newline at end of file diff --git a/vendor/go.temporal.io/sdk/client/client.go b/vendor/go.temporal.io/sdk/client/client.go new file mode 100644 index 00000000000..83d583fe346 --- /dev/null +++ b/vendor/go.temporal.io/sdk/client/client.go @@ -0,0 +1,1103 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +//go:generate mockgen -copyright_file ../LICENSE -package client -source client.go -destination client_mock.go + +// Package client is used by external programs to communicate with Temporal service. +// +// NOTE: DO NOT USE THIS API INSIDE OF ANY WORKFLOW CODE!!! +package client + +import ( + "context" + "crypto/tls" + "io" + + "go.temporal.io/api/cloud/cloudservice/v1" + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + historypb "go.temporal.io/api/history/v1" + "go.temporal.io/api/operatorservice/v1" + "go.temporal.io/api/workflowservice/v1" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal" + "go.temporal.io/sdk/internal/common/metrics" +) + +// TaskReachability specifies which category of tasks may reach a worker on a versioned task queue. +// Used both in a reachability query and its response. +// +// Deprecated: Use [BuildIDTaskReachability] +type TaskReachability = internal.TaskReachability + +const ( + // TaskReachabilityUnspecified indicates the reachability was not specified + TaskReachabilityUnspecified = internal.TaskReachabilityUnspecified + // TaskReachabilityNewWorkflows indicates the Build Id might be used by new workflows + TaskReachabilityNewWorkflows = internal.TaskReachabilityNewWorkflows + // TaskReachabilityExistingWorkflows indicates the Build Id might be used by open workflows + // and/or closed workflows. + TaskReachabilityExistingWorkflows = internal.TaskReachabilityExistingWorkflows + // TaskReachabilityOpenWorkflows indicates the Build Id might be used by open workflows. + TaskReachabilityOpenWorkflows = internal.TaskReachabilityOpenWorkflows + // TaskReachabilityClosedWorkflows indicates the Build Id might be used by closed workflows + TaskReachabilityClosedWorkflows = internal.TaskReachabilityClosedWorkflows +) + +// TaskQueueType specifies which category of tasks are associated with a queue. +// WARNING: Worker versioning is currently experimental +type TaskQueueType = internal.TaskQueueType + +const ( + // TaskQueueTypeUnspecified indicates the task queue type was not specified. + TaskQueueTypeUnspecified = internal.TaskQueueTypeUnspecified + // TaskQueueTypeWorkflow indicates the task queue is used for dispatching workflow tasks. + TaskQueueTypeWorkflow = internal.TaskQueueTypeWorkflow + // TaskQueueTypeActivity indicates the task queue is used for delivering activity tasks. + TaskQueueTypeActivity = internal.TaskQueueTypeActivity + // TaskQueueTypeNexus indicates the task queue is used for dispatching Nexus requests. + TaskQueueTypeNexus = internal.TaskQueueTypeNexus +) + +// BuildIDTaskReachability specifies which category of tasks may reach a versioned worker of a certain Build ID. +// Note: future activities who inherit their workflow's Build ID but not its task queue will not be +// accounted for reachability as server cannot know if they'll happen as they do not use +// assignment rules of their task queue. Same goes for Child Workflows or Continue-As-New Workflows +// who inherit the parent/previous workflow's Build ID but not its task queue. In those cases, make +// sure to query reachability for the parent/previous workflow's task queue as well. +// WARNING: Worker versioning is currently experimental +type BuildIDTaskReachability = internal.BuildIDTaskReachability + +const ( + // BuildIDTaskReachabilityUnspecified indicates that task reachability was not reported. + BuildIDTaskReachabilityUnspecified = internal.BuildIDTaskReachabilityUnspecified + // BuildIDTaskReachabilityReachable indicates that this Build ID may be used by new workflows or activities + // (based on versioning rules), or there are open workflows or backlogged activities assigned to it. + BuildIDTaskReachabilityReachable = internal.BuildIDTaskReachabilityReachable + // BuildIDTaskReachabilityClosedWorkflowsOnly specifies that this Build ID does not have open workflows + // and is not reachable by new workflows, but MAY have closed workflows within the namespace retention period. + // Not applicable to activity-only task queues. + BuildIDTaskReachabilityClosedWorkflowsOnly = internal.BuildIDTaskReachabilityClosedWorkflowsOnly + // BuildIDTaskReachabilityUnreachable indicates that this Build ID is not used for new executions, nor + // it has been used by any existing execution within the retention period. + BuildIDTaskReachabilityUnreachable = internal.BuildIDTaskReachabilityUnreachable +) + +// WorkflowUpdateStage indicates the stage of an update request. +// NOTE: Experimental +type WorkflowUpdateStage = internal.WorkflowUpdateStage + +const ( + // WorkflowUpdateStageUnspecified indicates the wait stage was not specified + // NOTE: Experimental + WorkflowUpdateStageUnspecified = internal.WorkflowUpdateStageUnspecified + // WorkflowUpdateStageAdmitted indicates the update is admitted + // NOTE: Experimental + WorkflowUpdateStageAdmitted = internal.WorkflowUpdateStageAdmitted + // WorkflowUpdateStageAccepted indicates the update is accepted + // NOTE: Experimental + WorkflowUpdateStageAccepted = internal.WorkflowUpdateStageAccepted + // WorkflowUpdateStageCompleted indicates the update is completed + // NOTE: Experimental + WorkflowUpdateStageCompleted = internal.WorkflowUpdateStageCompleted +) + +const ( + // DefaultHostPort is the host:port which is used if not passed with options. + DefaultHostPort = internal.LocalHostPort + + // DefaultNamespace is the namespace name which is used if not passed with options. + DefaultNamespace = internal.DefaultNamespace + + // QueryTypeStackTrace is the build in query type for Client.QueryWorkflow() call. Use this query type to get the call + // stack of the workflow. The result will be a string encoded in the converter.EncodedValue. + QueryTypeStackTrace string = internal.QueryTypeStackTrace + + // QueryTypeOpenSessions is the build in query type for Client.QueryWorkflow() call. Use this query type to get all open + // sessions in the workflow. The result will be a list of SessionInfo encoded in the converter.EncodedValue. + QueryTypeOpenSessions string = internal.QueryTypeOpenSessions + + // UnversionedBuildID is a stand-in for a Build Id for unversioned Workers. + // WARNING: Worker versioning is currently experimental + UnversionedBuildID string = internal.UnversionedBuildID +) + +type ( + // Options are optional parameters for Client creation. + Options = internal.ClientOptions + + // CloudOperationsClientOptions are parameters for CloudOperationsClient creation. + // + // WARNING: Cloud operations client is currently experimental. + CloudOperationsClientOptions = internal.CloudOperationsClientOptions + + // ConnectionOptions are optional parameters that can be specified in ClientOptions + ConnectionOptions = internal.ConnectionOptions + + // Credentials are optional credentials that can be specified in ClientOptions. + Credentials = internal.Credentials + + // StartWorkflowOptions configuration parameters for starting a workflow execution. + StartWorkflowOptions = internal.StartWorkflowOptions + + // WithStartWorkflowOperation is a type of operation that can be executed as part of a workflow start. + // For example, use NewUpdateWithStartWorkflowOperation to perform Update-with-Start. + // NOTE: Experimental + WithStartWorkflowOperation = internal.WithStartWorkflowOperation + + // UpdateWithStartWorkflowOperation is used to perform Update-with-Start. + // See NewUpdateWithStartWorkflowOperation for details. + // NOTE: Experimental + UpdateWithStartWorkflowOperation = internal.UpdateWithStartWorkflowOperation + + // HistoryEventIterator is a iterator which can return history events. + HistoryEventIterator = internal.HistoryEventIterator + + // WorkflowRun represents a started non child workflow. + WorkflowRun = internal.WorkflowRun + + // WorkflowRunGetOptions are options for WorkflowRun.GetWithOptions. + WorkflowRunGetOptions = internal.WorkflowRunGetOptions + + // QueryWorkflowWithOptionsRequest defines the request to QueryWorkflowWithOptions. + QueryWorkflowWithOptionsRequest = internal.QueryWorkflowWithOptionsRequest + + // QueryWorkflowWithOptionsResponse defines the response to QueryWorkflowWithOptions. + QueryWorkflowWithOptionsResponse = internal.QueryWorkflowWithOptionsResponse + + // CheckHealthRequest is a request for Client.CheckHealth. + CheckHealthRequest = internal.CheckHealthRequest + + // CheckHealthResponse is a response for Client.CheckHealth. + CheckHealthResponse = internal.CheckHealthResponse + + // ScheduleRange represents a set of integer values. + ScheduleRange = internal.ScheduleRange + + // ScheduleCalendarSpec is an event specification relative to the calendar. + ScheduleCalendarSpec = internal.ScheduleCalendarSpec + + // ScheduleIntervalSpec describes periods a schedules action should occur. + ScheduleIntervalSpec = internal.ScheduleIntervalSpec + + // ScheduleSpec describes when a schedules action should occur. + ScheduleSpec = internal.ScheduleSpec + + // SchedulePolicies describes the current polcies of a schedule. + SchedulePolicies = internal.SchedulePolicies + + // ScheduleState describes the current state of a schedule. + ScheduleState = internal.ScheduleState + + // ScheduleBackfill desribes a time periods and policy and takes Actions as if that time passed by right now, all at once. + ScheduleBackfill = internal.ScheduleBackfill + + // ScheduleAction is the interface for all actions a schedule can take. + ScheduleAction = internal.ScheduleAction + + // ScheduleWorkflowAction is the implementation of ScheduleAction to start a workflow. + ScheduleWorkflowAction = internal.ScheduleWorkflowAction + + // ScheduleOptions configuration parameters for creating a schedule. + ScheduleOptions = internal.ScheduleOptions + + // ScheduleClient is the interface with the server to create and get handles to schedules. + ScheduleClient = internal.ScheduleClient + + // ScheduleListOptions are configuration parameters for listing schedules. + ScheduleListOptions = internal.ScheduleListOptions + + // ScheduleListIterator is a iterator which can return created schedules. + ScheduleListIterator = internal.ScheduleListIterator + + // ScheduleListEntry is a result from ScheduleListEntry. + ScheduleListEntry = internal.ScheduleListEntry + + // ScheduleUpdateOptions are configuration parameters for updating a schedule. + ScheduleUpdateOptions = internal.ScheduleUpdateOptions + + // ScheduleHandle represents a created schedule. + ScheduleHandle = internal.ScheduleHandle + + // ScheduleActionResult describes when a schedule action took place. + ScheduleActionResult = internal.ScheduleActionResult + + // ScheduleWorkflowExecution contains details on a workflows execution stared by a schedule. + ScheduleWorkflowExecution = internal.ScheduleWorkflowExecution + + // ScheduleDescription describes the current Schedule details from ScheduleHandle.Describe. + ScheduleDescription = internal.ScheduleDescription + + // Schedule describes a created schedule. + Schedule = internal.Schedule + + // ScheduleUpdate describes the desired new schedule from ScheduleHandle.Update. + ScheduleUpdate = internal.ScheduleUpdate + + // ScheduleUpdateInput describes the current state of the schedule to be updated. + ScheduleUpdateInput = internal.ScheduleUpdateInput + + // ScheduleTriggerOptions configure the parameters for triggering a schedule. + ScheduleTriggerOptions = internal.ScheduleTriggerOptions + + // SchedulePauseOptions configure the parameters for pausing a schedule. + SchedulePauseOptions = internal.SchedulePauseOptions + + // ScheduleUnpauseOptions configure the parameters for unpausing a schedule. + ScheduleUnpauseOptions = internal.ScheduleUnpauseOptions + + // ScheduleBackfillOptions configure the parameters for backfilling a schedule. + ScheduleBackfillOptions = internal.ScheduleBackfillOptions + + // UpdateWorkflowOptions encapsulates the parameters for + // sending an update to a workflow execution. + // NOTE: Experimental + UpdateWorkflowOptions = internal.UpdateWorkflowOptions + + // WorkflowUpdateHandle represents a running or completed workflow + // execution update and gives the holder access to the outcome of the same. + // NOTE: Experimental + WorkflowUpdateHandle = internal.WorkflowUpdateHandle + + // GetWorkflowUpdateHandleOptions encapsulates the parameters needed to unambiguously + // refer to a Workflow Update + // NOTE: Experimental + GetWorkflowUpdateHandleOptions = internal.GetWorkflowUpdateHandleOptions + + // UpdateWorkerBuildIdCompatibilityOptions is the input to Client.UpdateWorkerBuildIdCompatibility. + // + // Deprecated: Use [UpdateWorkerVersioningRulesOptions] with the new worker versioning api. + UpdateWorkerBuildIdCompatibilityOptions = internal.UpdateWorkerBuildIdCompatibilityOptions + + // GetWorkerBuildIdCompatibilityOptions is the input to Client.GetWorkerBuildIdCompatibility. + // + // Deprecated: Use [GetWorkerVersioningOptions] with the new worker versioning api. + GetWorkerBuildIdCompatibilityOptions = internal.GetWorkerBuildIdCompatibilityOptions + + // WorkerBuildIDVersionSets is the response for Client.GetWorkerBuildIdCompatibility. + // + // Deprecated: Replaced by the new worker versioning api. + WorkerBuildIDVersionSets = internal.WorkerBuildIDVersionSets + + // BuildIDOpAddNewIDInNewDefaultSet is an operation for UpdateWorkerBuildIdCompatibilityOptions + // to add a new BuildID in a new default set. + // + // Deprecated: Replaced by the new worker versioning api. + BuildIDOpAddNewIDInNewDefaultSet = internal.BuildIDOpAddNewIDInNewDefaultSet + + // BuildIDOpAddNewCompatibleVersion is an operation for UpdateWorkerBuildIdCompatibilityOptions + // to add a new BuildID to an existing compatible set. + // + // Deprecated: Replaced by the new worker versioning api. + BuildIDOpAddNewCompatibleVersion = internal.BuildIDOpAddNewCompatibleVersion + + // BuildIDOpPromoteSet is an operation for UpdateWorkerBuildIdCompatibilityOptions to promote a + // set to be the default set by targeting an existing BuildID. + // + // Deprecated: Replaced by the new worker versioning api. + BuildIDOpPromoteSet = internal.BuildIDOpPromoteSet + + // BuildIDOpPromoteIDWithinSet is an operation for UpdateWorkerBuildIdCompatibilityOptions to + // promote a BuildID within a set to be the default. + // + // Deprecated: Replaced by the new worker versioning api. + BuildIDOpPromoteIDWithinSet = internal.BuildIDOpPromoteIDWithinSet + + // GetWorkerTaskReachabilityOptions is the input to Client.GetWorkerTaskReachability. + // + // Deprecated: Use [DescribeTaskQueueEnhancedOptions] with the new worker versioning api. + GetWorkerTaskReachabilityOptions = internal.GetWorkerTaskReachabilityOptions + + // WorkerTaskReachability is the response for Client.GetWorkerTaskReachability. + // + // Deprecated: Replaced by the new worker versioning api. + WorkerTaskReachability = internal.WorkerTaskReachability + + // BuildIDReachability describes the reachability of a buildID + // + // Deprecated: Replaced by the new worker versioning api. + BuildIDReachability = internal.BuildIDReachability + + // TaskQueueReachability Describes how the Build ID may be reachable from the task queue. + // + // Deprecated: Replaced by the new worker versioning api. + TaskQueueReachability = internal.TaskQueueReachability + + // DescribeTaskQueueEnhancedOptions is the input to [Client.DescribeTaskQueueEnhanced]. + DescribeTaskQueueEnhancedOptions = internal.DescribeTaskQueueEnhancedOptions + + // TaskQueueVersionSelection is a task queue filter based on versioning. + // It is an optional component of [Client.DescribeTaskQueueEnhancedOptions]. + // WARNING: Worker versioning is currently experimental. + TaskQueueVersionSelection = internal.TaskQueueVersionSelection + + // TaskQueueDescription is the response to [Client.DescribeTaskQueueEnhanced]. + TaskQueueDescription = internal.TaskQueueDescription + + // TaskQueueVersionInfo includes task queue information per Build ID. + // It is part of [Client.TaskQueueDescription]. + TaskQueueVersionInfo = internal.TaskQueueVersionInfo + + // TaskQueueTypeInfo specifies task queue information per task type and Build ID. + // It is included in [Client.TaskQueueVersionInfo]. + TaskQueueTypeInfo = internal.TaskQueueTypeInfo + + // TaskQueuePollerInfo provides information about a worker/client polling a task queue. + // It is used by [Client.TaskQueueTypeInfo]. + TaskQueuePollerInfo = internal.TaskQueuePollerInfo + + // TaskQueueStats contains statistics about task queue backlog and activity. + // + // For workflow task queue type, this result is partial because tasks sent to sticky queues are not included. Read + // comments above each metric to understand the impact of sticky queue exclusion on that metric accuracy. + TaskQueueStats = internal.TaskQueueStats + + // WorkerVersionCapabilities includes a worker's build identifier + // and whether it is choosing to use the versioning feature. + // It is an optional component of [Client.TaskQueuePollerInfo]. + // WARNING: Worker versioning is currently experimental. + WorkerVersionCapabilities = internal.WorkerVersionCapabilities + + // UpdateWorkerVersioningRulesOptions is the input to [Client.UpdateWorkerVersioningRules]. + // WARNING: Worker versioning is currently experimental. + UpdateWorkerVersioningRulesOptions = internal.UpdateWorkerVersioningRulesOptions + + // VersioningConflictToken is a conflict token to serialize calls to Client.UpdateWorkerVersioningRules. + // An update with an old token fails with `serviceerror.FailedPrecondition`. + // The current token can be obtained with [GetWorkerVersioningRules], + // or returned by a successful [UpdateWorkerVersioningRules]. + // WARNING: Worker versioning is currently experimental. + VersioningConflictToken = internal.VersioningConflictToken + + // VersioningRampByPercentage is a VersionRamp that sends a proportion of the traffic + // to the target Build ID. + // WARNING: Worker versioning is currently experimental. + VersioningRampByPercentage = internal.VersioningRampByPercentage + + // VersioningAssignmentRule is a BuildID assigment rule for a task queue. + // Assignment rules only affect new workflows. + // WARNING: Worker versioning is currently experimental. + VersioningAssignmentRule = internal.VersioningAssignmentRule + + // VersioningAssignmentRuleWithTimestamp contains an assignment rule annotated + // by the server with its creation time. + // WARNING: Worker versioning is currently experimental. + VersioningAssignmentRuleWithTimestamp = internal.VersioningAssignmentRuleWithTimestamp + + // VersioningAssignmentRule is a BuildID redirect rule for a task queue. + // It changes the behavior of currently running workflows and new ones. + // WARNING: Worker versioning is currently experimental. + VersioningRedirectRule = internal.VersioningRedirectRule + + // VersioningRedirectRuleWithTimestamp contains a redirect rule annotated + // by the server with its creation time. + // WARNING: Worker versioning is currently experimental. + VersioningRedirectRuleWithTimestamp = internal.VersioningRedirectRuleWithTimestamp + + // VersioningOperationInsertAssignmentRule is an operation for UpdateWorkerVersioningRulesOptions + // that inserts the rule to the list of assignment rules for this Task Queue. + // The rules are evaluated in order, starting from index 0. The first + // applicable rule will be applied and the rest will be ignored. + // By default, the new rule is inserted at the beginning of the list + // (index 0). If the given index is too larger the rule will be + // inserted at the end of the list. + // WARNING: Worker versioning is currently experimental. + VersioningOperationInsertAssignmentRule = internal.VersioningOperationInsertAssignmentRule + + // VersioningOperationReplaceAssignmentRule is an operation for UpdateWorkerVersioningRulesOptions + // that replaces the assignment rule at a given index. By default presence of one + // unconditional rule, i.e., no hint filter or ramp, is enforced, otherwise + // the delete operation will be rejected. Set `force` to true to + // bypass this validation. + // WARNING: Worker versioning is currently experimental. + VersioningOperationReplaceAssignmentRule = internal.VersioningOperationReplaceAssignmentRule + + // VersioningOperationDeleteAssignmentRule is an operation for UpdateWorkerVersioningRulesOptions + // that deletes the assignment rule at a given index. By default presence of one + // unconditional rule, i.e., no hint filter or ramp, is enforced, otherwise + // the delete operation will be rejected. Set `force` to true to + // bypass this validation. + // WARNING: Worker versioning is currently experimental. + VersioningOperationDeleteAssignmentRule = internal.VersioningOperationDeleteAssignmentRule + + // VersioningOperationAddRedirectRule is an operation for UpdateWorkerVersioningRulesOptions + // that adds the rule to the list of redirect rules for this Task Queue. There + // can be at most one redirect rule for each distinct Source BuildID. + // WARNING: Worker versioning is currently experimental. + VersioningOperationAddRedirectRule = internal.VersioningOperationAddRedirectRule + + // VersioningOperationReplaceRedirectRule is an operation for UpdateWorkerVersioningRulesOptions + // that replaces the routing rule with the given source BuildID. + // WARNING: Worker versioning is currently experimental. + VersioningOperationReplaceRedirectRule = internal.VersioningOperationReplaceRedirectRule + + // VersioningOperationDeleteRedirectRule is an operation for UpdateWorkerVersioningRulesOptions + // that deletes the routing rule with the given source Build ID. + // WARNING: Worker versioning is currently experimental. + VersioningOperationDeleteRedirectRule = internal.VersioningOperationDeleteRedirectRule + + // VersioningOperationCommitBuildID is an operation for UpdateWorkerVersioningRulesOptions + // that completes the rollout of a BuildID and cleanup unnecessary rules possibly + // created during a gradual rollout. Specifically, this command will make the following changes + // atomically: + // 1. Adds an assignment rule (with full ramp) for the target Build ID at + // the end of the list. + // 2. Removes all previously added assignment rules to the given target + // Build ID (if any). + // 3. Removes any fully-ramped assignment rule for other Build IDs. + // + // To prevent committing invalid Build IDs, we reject the request if no + // pollers have been seen recently for this Build ID. Use the `force` + // option to disable this validation. + // WARNING: Worker versioning is currently experimental. + VersioningOperationCommitBuildID = internal.VersioningOperationCommitBuildID + + // GetWorkerVersioningOptions is the input to [Client.GetWorkerVersioningRules]. + // WARNING: Worker versioning is currently experimental. + GetWorkerVersioningOptions = internal.GetWorkerVersioningOptions + + // WorkerVersioningRules is the response for [Client.GetWorkerVersioningRules]. + // WARNING: Worker versioning is currently experimental. + WorkerVersioningRules = internal.WorkerVersioningRules + + // WorkflowUpdateServiceTimeoutOrCanceledError is an error that occurs when an update call times out or is cancelled. + // + // Note, this is not related to any general concept of timing out or cancelling a running update, this is only related to the client call itself. + // NOTE: Experimental + WorkflowUpdateServiceTimeoutOrCanceledError = internal.WorkflowUpdateServiceTimeoutOrCanceledError + + // Client is the client for starting and getting information about a workflow executions as well as + // completing activities asynchronously. + Client interface { + // ExecuteWorkflow starts a workflow execution and returns a WorkflowRun instance or error + // + // This can be used to start a workflow using a function reference or workflow type name. + // Either by + // ExecuteWorkflow(ctx, options, "workflowTypeName", arg1, arg2, arg3) + // or + // ExecuteWorkflow(ctx, options, workflowExecuteFn, arg1, arg2, arg3) + // The errors it can return: + // - serviceerror.NamespaceNotFound, if namespace does not exist + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // + // WorkflowRun has 3 methods: + // - GetWorkflowID() string: which return the started workflow ID + // - GetRunID() string: which return the first started workflow run ID (please see below) + // - Get(ctx context.Context, valuePtr interface{}) error: which will fill the workflow + // execution result to valuePtr, if workflow execution is a success, or return corresponding + // error. This is a blocking API. + // NOTE: if the started workflow returns ContinueAsNewError during the workflow execution, the + // returned result of GetRunID() will be the started workflow run ID, not the new run ID caused by ContinueAsNewError, + // however, Get(ctx context.Context, valuePtr interface{}) will return result from the run which did not return ContinueAsNewError. + // Say ExecuteWorkflow started a workflow, in its first run, has run ID "run ID 1", and returned ContinueAsNewError, + // the second run has run ID "run ID 2" and return some result other than ContinueAsNewError: + // GetRunID() will always return "run ID 1" and Get(ctx context.Context, valuePtr interface{}) will return the result of second run. + // NOTE: DO NOT USE THIS API INSIDE A WORKFLOW, USE workflow.ExecuteChildWorkflow instead + ExecuteWorkflow(ctx context.Context, options StartWorkflowOptions, workflow interface{}, args ...interface{}) (WorkflowRun, error) + + // GetWorkflow retrieves a workflow execution and return a WorkflowRun instance (described above) + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. + // + // WorkflowRun has 2 methods: + // - GetRunID() string: which return the first started workflow run ID (please see below) + // - Get(ctx context.Context, valuePtr interface{}) error: which will fill the workflow + // execution result to valuePtr, if workflow execution is a success, or return corresponding + // error. This is a blocking API. + // If workflow not found, the Get() will return serviceerror.NotFound. + // NOTE: if the started workflow return ContinueAsNewError during the workflow execution, the + // return result of GetRunID() will be the started workflow run ID, not the new run ID caused by ContinueAsNewError, + // however, Get(ctx context.Context, valuePtr interface{}) will return result from the run which did not return ContinueAsNewError. + // Say ExecuteWorkflow started a workflow, in its first run, has run ID "run ID 1", and returned ContinueAsNewError, + // the second run has run ID "run ID 2" and return some result other than ContinueAsNewError: + // GetRunID() will always return "run ID 1" and Get(ctx context.Context, valuePtr interface{}) will return the result of second run. + GetWorkflow(ctx context.Context, workflowID string, runID string) WorkflowRun + + // SignalWorkflow sends a signals to a workflow in execution + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // - signalName name to identify the signal. + // The errors it can return: + // - serviceerror.NotFound + // - serviceerror.Internal + // - serviceerror.Unavailable + SignalWorkflow(ctx context.Context, workflowID string, runID string, signalName string, arg interface{}) error + + // SignalWithStartWorkflow sends a signal to a running workflow. + // If the workflow is not running or not found, it starts the workflow and then sends the signal in transaction. + // - workflowID, signalName, signalArg are same as SignalWorkflow's parameters + // - options, workflow, workflowArgs are same as StartWorkflow's parameters + // - the workflowID parameter is used instead of options.ID. If the latter is present, it must match the workflowID. + // Note: options.WorkflowIDReusePolicy is default to AllowDuplicate in this API. + // The errors it can return: + // - serviceerror.NotFound + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + SignalWithStartWorkflow(ctx context.Context, workflowID string, signalName string, signalArg interface{}, + options StartWorkflowOptions, workflow interface{}, workflowArgs ...interface{}) (WorkflowRun, error) + + // CancelWorkflow request cancellation of a workflow in execution. Cancellation request closes the channel + // returned by the workflow.Context.Done() of the workflow that is target of the request. + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the currently running execution of that workflow ID. + // The errors it can return: + // - serviceerror.NotFound + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + CancelWorkflow(ctx context.Context, workflowID string, runID string) error + + // TerminateWorkflow terminates a workflow execution. Terminate stops a workflow execution immediately without + // letting the workflow to perform any cleanup + // workflowID is required, other parameters are optional. + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // The errors it can return: + // - serviceerror.NotFound + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + TerminateWorkflow(ctx context.Context, workflowID string, runID string, reason string, details ...interface{}) error + + // GetWorkflowHistory gets history events of a particular workflow + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. + // - whether use long poll for tracking new events: when the workflow is running, there can be new events generated during iteration + // of HistoryEventIterator, if isLongPoll == true, then iterator will do long poll, tracking new history event, i.e. the iteration + // will not be finished until workflow is finished; if isLongPoll == false, then iterator will only return current history events. + // - whether return all history events or just the last event, which contains the workflow execution end result + // Example:- + // To iterate all events, + // iter := GetWorkflowHistory(ctx, workflowID, runID, isLongPoll, filterType) + // events := []*shared.HistoryEvent{} + // for iter.HasNext() { + // event, err := iter.Next() + // if err != nil { + // return err + // } + // events = append(events, event) + // } + GetWorkflowHistory(ctx context.Context, workflowID string, runID string, isLongPoll bool, filterType enumspb.HistoryEventFilterType) HistoryEventIterator + + // CompleteActivity reports activity completed. + // activity Execute method can return activity.ErrResultPending to + // indicate the activity is not completed when it's Execute method returns. In that case, this CompleteActivity() method + // should be called when that activity is completed with the actual result and error. If err is nil, activity task + // completed event will be reported; if err is CanceledError, activity task canceled event will be reported; otherwise, + // activity task failed event will be reported. + // An activity implementation should use GetActivityInfo(ctx).TaskToken function to get task token to use for completion. + // Example:- + // To complete with a result. + // CompleteActivity(token, "Done", nil) + // To fail the activity with an error. + // CompleteActivity(token, nil, temporal.NewApplicationError("reason", details) + // The activity can fail with below errors ApplicationError, TimeoutError, CanceledError. + CompleteActivity(ctx context.Context, taskToken []byte, result interface{}, err error) error + + // CompleteActivityByID reports activity completed. + // Similar to CompleteActivity, but may save user from keeping taskToken info. + // activity Execute method can return activity.ErrResultPending to + // indicate the activity is not completed when it's Execute method returns. In that case, this CompleteActivityById() method + // should be called when that activity is completed with the actual result and error. If err is nil, activity task + // completed event will be reported; if err is CanceledError, activity task canceled event will be reported; otherwise, + // activity task failed event will be reported. + // An activity implementation should use activityID provided in ActivityOption to use for completion. + // namespace name, workflowID, activityID are required, runID is optional. + // The errors it can return: + // - ApplicationError + // - TimeoutError + // - CanceledError + CompleteActivityByID(ctx context.Context, namespace, workflowID, runID, activityID string, result interface{}, err error) error + + // RecordActivityHeartbeat records heartbeat for an activity. + // taskToken - is the value of the binary "TaskToken" field of the "ActivityInfo" struct retrieved inside the activity. + // details - is the progress you want to record along with heart beat for this activity. + // The errors it can return: + // - serviceerror.NotFound + // - serviceerror.Internal + // - serviceerror.Unavailable + RecordActivityHeartbeat(ctx context.Context, taskToken []byte, details ...interface{}) error + + // RecordActivityHeartbeatByID records heartbeat for an activity. + // details - is the progress you want to record along with heart beat for this activity. + // The errors it can return: + // - serviceerror.NotFound + // - serviceerror.Internal + // - serviceerror.Unavailable + RecordActivityHeartbeatByID(ctx context.Context, namespace, workflowID, runID, activityID string, details ...interface{}) error + + // ListClosedWorkflow gets closed workflow executions based on request filters. + // Retrieved workflow executions are sorted by close time in descending order. + // Note: heavy usage of this API may cause huge persistence pressure. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NamespaceNotFound + ListClosedWorkflow(ctx context.Context, request *workflowservice.ListClosedWorkflowExecutionsRequest) (*workflowservice.ListClosedWorkflowExecutionsResponse, error) + + // ListOpenWorkflow gets open workflow executions based on request filters. + // Retrieved workflow executions are sorted by start time in descending order. + // Note: heavy usage of this API may cause huge persistence pressure. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NamespaceNotFound + ListOpenWorkflow(ctx context.Context, request *workflowservice.ListOpenWorkflowExecutionsRequest) (*workflowservice.ListOpenWorkflowExecutionsResponse, error) + + // ListWorkflow gets workflow executions based on query. The query is basically the SQL WHERE clause, examples: + // - "(WorkflowID = 'wid1' or (WorkflowType = 'type2' and WorkflowID = 'wid2'))". + // - "CloseTime between '2019-08-27T15:04:05+00:00' and '2019-08-28T15:04:05+00:00'". + // - to list only open workflow use "CloseTime is null" + // For supported operations on different server versions see https://docs.temporal.io/visibility. + // Retrieved workflow executions are sorted by StartTime in descending order when list open workflow, + // and sorted by CloseTime in descending order for other queries. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + ListWorkflow(ctx context.Context, request *workflowservice.ListWorkflowExecutionsRequest) (*workflowservice.ListWorkflowExecutionsResponse, error) + + // ListArchivedWorkflow gets archived workflow executions based on query. This API will return BadRequest if Temporal + // cluster or target namespace is not configured for visibility archival or read is not enabled. The query is basically the SQL WHERE clause. + // However, different visibility archivers have different limitations on the query. Please check the documentation of the visibility archiver used + // by your namespace to see what kind of queries are accept and whether retrieved workflow executions are ordered or not. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + ListArchivedWorkflow(ctx context.Context, request *workflowservice.ListArchivedWorkflowExecutionsRequest) (*workflowservice.ListArchivedWorkflowExecutionsResponse, error) + + // ScanWorkflow gets workflow executions based on query. The query is basically the SQL WHERE clause + // (see ListWorkflow for query examples). + // For supported operations on different server versions see https://docs.temporal.io/visibility. + // ScanWorkflow should be used when retrieving large amount of workflows and order is not needed. + // It will use more resources than ListWorkflow, but will be several times faster + // when retrieving millions of workflows. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + ScanWorkflow(ctx context.Context, request *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error) + + // CountWorkflow gets number of workflow executions based on query. The query is basically the SQL WHERE clause + // (see ListWorkflow for query examples). + // For supported operations on different server versions see https://docs.temporal.io/visibility. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + CountWorkflow(ctx context.Context, request *workflowservice.CountWorkflowExecutionsRequest) (*workflowservice.CountWorkflowExecutionsResponse, error) + + // GetSearchAttributes returns valid search attributes keys and value types. + // The search attributes can be used in query of List/Scan/Count APIs. Adding new search attributes requires temporal server + // to update dynamic config ValidSearchAttributes. + // NOTE: This API is not supported on Temporal Cloud. + GetSearchAttributes(ctx context.Context) (*workflowservice.GetSearchAttributesResponse, error) + + // QueryWorkflow queries a given workflow's last execution and returns the query result synchronously. Parameter workflowID + // and queryType are required, other parameters are optional. The workflowID and runID (optional) identify the + // target workflow execution that this query will be send to. If runID is not specified (empty string), server will + // use the currently running execution of that workflowID. The queryType specifies the type of query you want to + // run. By default, temporal supports "__stack_trace" as a standard query type, which will return string value + // representing the call stack of the target workflow. The target workflow could also setup different query handler + // to handle custom query types. + // See comments at workflow.SetQueryHandler(ctx Context, queryType string, handler interface{}) for more details + // on how to setup query handler within the target workflow. + // - workflowID is required. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // - queryType is the type of the query. + // - args... are the optional query parameters. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NotFound + // - serviceerror.QueryFailed + QueryWorkflow(ctx context.Context, workflowID string, runID string, queryType string, args ...interface{}) (converter.EncodedValue, error) + + // QueryWorkflowWithOptions queries a given workflow execution and returns the query result synchronously. + // See QueryWorkflowWithOptionsRequest and QueryWorkflowWithOptionsResponse for more information. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NotFound + // - serviceerror.QueryFailed + QueryWorkflowWithOptions(ctx context.Context, request *QueryWorkflowWithOptionsRequest) (*QueryWorkflowWithOptionsResponse, error) + + // DescribeWorkflowExecution returns information about the specified workflow execution. + // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. + // + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NotFound + DescribeWorkflowExecution(ctx context.Context, workflowID, runID string) (*workflowservice.DescribeWorkflowExecutionResponse, error) + + // DescribeTaskQueue returns information about the target taskqueue, right now this API returns the + // pollers which polled this taskqueue in last few minutes. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NotFound + DescribeTaskQueue(ctx context.Context, taskqueue string, taskqueueType enumspb.TaskQueueType) (*workflowservice.DescribeTaskQueueResponse, error) + + // DescribeTaskQueueEnhanced returns information about the target task queue, broken down by Build Id: + // - List of pollers + // - Workflow Reachability status + // - Backlog info for Workflow and/or Activity tasks + // When not supported by the server, it returns an empty [TaskQueueDescription] if there is no information + // about the task queue, or an error when the response identifies an unsupported server. + // Note that using a sticky queue as target is not supported. + // Also, workflow reachability status is eventually consistent, and it could take a few minutes to update. + // WARNING: Worker versioning is currently experimental, and requires server 1.24+ + DescribeTaskQueueEnhanced(ctx context.Context, options DescribeTaskQueueEnhancedOptions) (TaskQueueDescription, error) + + // ResetWorkflowExecution resets an existing workflow execution to WorkflowTaskFinishEventId(exclusive). + // And it will immediately terminating the current execution instance. + // RequestId is used to deduplicate requests. It will be autogenerated if not set. + ResetWorkflowExecution(ctx context.Context, request *workflowservice.ResetWorkflowExecutionRequest) (*workflowservice.ResetWorkflowExecutionResponse, error) + + // UpdateWorkerBuildIdCompatibility + // Allows you to update the worker-build-id based version sets for a particular task queue. This is used in + // conjunction with workers who specify their build id and thus opt into the feature. + // + // Deprecated: Use [UpdateWorkerVersioningRules] with the versioning api. + UpdateWorkerBuildIdCompatibility(ctx context.Context, options *UpdateWorkerBuildIdCompatibilityOptions) error + + // GetWorkerBuildIdCompatibility + // Returns the worker-build-id based version sets for a particular task queue. + // + // Deprecated: Use [GetWorkerVersioningRules] with the versioning api. + GetWorkerBuildIdCompatibility(ctx context.Context, options *GetWorkerBuildIdCompatibilityOptions) (*WorkerBuildIDVersionSets, error) + + // GetWorkerTaskReachability + // Returns which versions are is still in use by open or closed workflows + // + // Deprecated: Use [DescribeTaskQueueEnhanced] with the versioning api. + GetWorkerTaskReachability(ctx context.Context, options *GetWorkerTaskReachabilityOptions) (*WorkerTaskReachability, error) + + // UpdateWorkerVersioningRules + // Allows updating the worker-build-id based assignment and redirect rules for a given task queue. This is used in + // conjunction with workers who specify their build id and thus opt into the feature. + // The errors it can return: + // - serviceerror.FailedPrecondition when the conflict token is invalid + // WARNING: Worker versioning is currently experimental, and requires server 1.24+ + UpdateWorkerVersioningRules(ctx context.Context, options UpdateWorkerVersioningRulesOptions) (*WorkerVersioningRules, error) + + // GetWorkerVersioningRules + // Returns the worker-build-id assignment and redirect rules for a task queue. + // WARNING: Worker versioning is currently experimental, and requires server 1.24+ + GetWorkerVersioningRules(ctx context.Context, options GetWorkerVersioningOptions) (*WorkerVersioningRules, error) + + // CheckHealth performs a server health check using the gRPC health check + // API. If the check fails, an error is returned. + CheckHealth(ctx context.Context, request *CheckHealthRequest) (*CheckHealthResponse, error) + + // UpdateWorkflow issues an update request to the + // specified workflow execution and returns a handle to the update that + // is running in in parallel with the calling thread. Errors returned + // from the server will be exposed through the return value of + // WorkflowUpdateHandle.Get(). Errors that occur before the + // update is requested (e.g. if the required workflow ID field is + // missing from the UpdateWorkflowOptions) are returned + // directly from this function call. + // + // The errors it can return: + // - WorkflowUpdateServiceTimeoutOrCanceledError + // NOTE: Experimental + UpdateWorkflow(ctx context.Context, options UpdateWorkflowOptions) (WorkflowUpdateHandle, error) + + // GetWorkflowUpdateHandle creates a handle to the referenced update + // which can be polled for an outcome. Note that runID is optional and + // if not specified the most recent runID will be used. + // NOTE: Experimental + GetWorkflowUpdateHandle(ref GetWorkflowUpdateHandleOptions) WorkflowUpdateHandle + + // WorkflowService provides access to the underlying gRPC service. This should only be used for advanced use cases + // that cannot be accomplished via other Client methods. Unlike calls to other Client methods, calls directly to the + // service are not configured with internal semantics such as automatic retries. + WorkflowService() workflowservice.WorkflowServiceClient + + // OperatorService creates a new operator service client with the same gRPC connection as this client. + OperatorService() operatorservice.OperatorServiceClient + + // Schedule creates a new shedule client with the same gRPC connection as this client. + ScheduleClient() ScheduleClient + + // Close client and clean up underlying resources. + // + // If this client was created via NewClientFromExisting or this client has + // been used in that call, Close() on may not necessarily close the + // underlying connection. Only the final close of all existing clients will + // close the underlying connection. + Close() + } + + // CloudOperationsClient is the client for cloud operations. + // + // WARNING: Cloud operations client is currently experimental. + CloudOperationsClient interface { + // CloudService provides access to the underlying gRPC service. + CloudService() cloudservice.CloudServiceClient + + // Close client and clean up underlying resources. + Close() + } + + // NamespaceClient is the client for managing operations on the namespace. + // CLI, tools, ... can use this layer to manager operations on namespace. + NamespaceClient interface { + // Register a namespace with temporal server + // The errors it can throw: + // - NamespaceAlreadyExistsError + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + Register(ctx context.Context, request *workflowservice.RegisterNamespaceRequest) error + + // Describe a namespace. The namespace has 3 part of information + // NamespaceInfo - Which has Name, Status, Description, Owner Email + // NamespaceConfiguration - Configuration like Workflow Execution Retention Period In Days, Whether to emit metrics. + // ReplicationConfiguration - replication config like clusters and active cluster name + // The errors it can throw: + // - serviceerror.NamespaceNotFound + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + Describe(ctx context.Context, name string) (*workflowservice.DescribeNamespaceResponse, error) + + // Update a namespace. + // The errors it can throw: + // - serviceerror.NamespaceNotFound + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + Update(ctx context.Context, request *workflowservice.UpdateNamespaceRequest) error + + // Close client and clean up underlying resources. + Close() + } +) + +// MetricsHandler is a handler for metrics emitted by the SDK. This interface is +// intentionally limited to only what the SDK needs to emit metrics and is not +// built to be a general purpose metrics abstraction for all uses. +// +// A common implementation is at +// go.temporal.io/sdk/contrib/tally.NewMetricsHandler. The MetricsNopHandler is +// a noop handler. A handler may implement "Unwrap() client.MetricsHandler" if +// it wraps a handler. +type MetricsHandler = metrics.Handler + +// MetricsCounter is an ever-increasing counter. +type MetricsCounter = metrics.Counter + +// MetricsGauge can be set to any float. +type MetricsGauge = metrics.Gauge + +// MetricsTimer records time durations. +type MetricsTimer = metrics.Timer + +// MetricsNopHandler is a noop handler that does nothing with the metrics. +var MetricsNopHandler = metrics.NopHandler + +// NewUpdateWithStartWorkflowOperation returns an UpdateWithStartWorkflowOperation to perform Update-with-Start. +// After executing Client.ExecuteWorkflow with the UpdateWithStartWorkflow in the start options, +// the update result can be obtained. +// NOTE: Experimental +func NewUpdateWithStartWorkflowOperation(options UpdateWorkflowOptions) *UpdateWithStartWorkflowOperation { + return internal.NewUpdateWithStartWorkflowOperation(options) +} + +// Dial creates an instance of a workflow client. This will attempt to connect +// to the server eagerly and will return an error if the server is not +// available. +func Dial(options Options) (Client, error) { + return DialContext(context.Background(), options) +} + +// DialContext creates an instance of a workflow client. This will attempt to connect +// to the server eagerly and will return an error if the server is not +// available. Connection will respect provided context deadlines and cancellations. +func DialContext(ctx context.Context, options Options) (Client, error) { + return internal.DialClient(ctx, options) +} + +// NewLazyClient creates an instance of a workflow client. Unlike Dial, this +// will not eagerly connect to the server. +func NewLazyClient(options Options) (Client, error) { + return internal.NewLazyClient(options) +} + +// NewClient creates an instance of a workflow client. This will attempt to +// connect to the server eagerly and will return an error if the server is not +// available. +// +// Deprecated: Use Dial or NewLazyClient instead. +func NewClient(options Options) (Client, error) { + return internal.NewClient(context.Background(), options) +} + +// NewClientFromExisting creates a new client using the same connection as the +// existing client. This means all options.ConnectionOptions are ignored and +// options.HostPort is ignored. The existing client must have been created from +// this package and cannot be wrapped. Currently, this always attempts an eager +// connection even if the existing client was created with NewLazyClient and has +// not made any calls yet. +// +// Close() on the resulting client may not necessarily close the underlying +// connection if there are any other clients using the connection. All clients +// associated with the existing client must call Close() and only the last one +// actually performs the connection close. +func NewClientFromExisting(existingClient Client, options Options) (Client, error) { + return NewClientFromExistingWithContext(context.Background(), existingClient, options) +} + +// NewClientFromExistingWithContext creates a new client using the same connection as the +// existing client. This means all options.ConnectionOptions are ignored and +// options.HostPort is ignored. The existing client must have been created from +// this package and cannot be wrapped. Currently, this always attempts an eager +// connection even if the existing client was created with NewLazyClient and has +// not made any calls yet. +// +// Close() on the resulting client may not necessarily close the underlying +// connection if there are any other clients using the connection. All clients +// associated with the existing client must call Close() and only the last one +// actually performs the connection close. +func NewClientFromExistingWithContext(ctx context.Context, existingClient Client, options Options) (Client, error) { + return internal.NewClientFromExisting(ctx, existingClient, options) +} + +// DialCloudOperationsClient creates a cloud client to perform cloud-management +// operations. Users should provide Credentials in the options. +// +// WARNING: Cloud operations client is currently experimental. +func DialCloudOperationsClient(ctx context.Context, options CloudOperationsClientOptions) (CloudOperationsClient, error) { + return internal.DialCloudOperationsClient(ctx, options) +} + +// NewNamespaceClient creates an instance of a namespace client, to manage +// lifecycle of namespaces. This will not attempt to connect to the server +// eagerly and therefore may not fail for an unreachable server until a call is +// made. +func NewNamespaceClient(options Options) (NamespaceClient, error) { + return internal.NewNamespaceClient(options) +} + +// make sure if new methods are added to internal.Client they are also added to public Client. +var ( + _ Client = internal.Client(nil) + _ internal.Client = Client(nil) + _ CloudOperationsClient = internal.CloudOperationsClient(nil) + _ internal.CloudOperationsClient = CloudOperationsClient(nil) + _ NamespaceClient = internal.NamespaceClient(nil) + _ internal.NamespaceClient = NamespaceClient(nil) +) + +// NewValue creates a new [converter.EncodedValue] which can be used to decode binary data returned by Temporal. For example: +// User had Activity.RecordHeartbeat(ctx, "my-heartbeat") and then got response from calling Client.DescribeWorkflowExecution. +// The response contains binary field PendingActivityInfo.HeartbeatDetails, +// which can be decoded by using: +// +// var result string // This need to be same type as the one passed to RecordHeartbeat +// NewValue(data).Get(&result) +func NewValue(data *commonpb.Payloads) converter.EncodedValue { + return internal.NewValue(data) +} + +// NewValues creates a new [converter.EncodedValues] which can be used to decode binary data returned by Temporal. For example: +// User had Activity.RecordHeartbeat(ctx, "my-heartbeat", 123) and then got response from calling Client.DescribeWorkflowExecution. +// The response contains binary field PendingActivityInfo.HeartbeatDetails, +// which can be decoded by using: +// +// var result1 string +// var result2 int // These need to be same type as those arguments passed to RecordHeartbeat +// NewValues(data).Get(&result1, &result2) +func NewValues(data *commonpb.Payloads) converter.EncodedValues { + return internal.NewValues(data) +} + +// HistoryJSONOptions are options for HistoryFromJSON. +type HistoryJSONOptions struct { + // LastEventID, if set, will only load history up to this ID (inclusive). + LastEventID int64 +} + +// HistoryFromJSON deserializes history from a reader of JSON bytes. This does +// not close the reader if it is closeable. +func HistoryFromJSON(r io.Reader, options HistoryJSONOptions) (*historypb.History, error) { + return internal.HistoryFromJSON(r, options.LastEventID) +} + +// NewAPIKeyStaticCredentials creates credentials that can be provided to +// ClientOptions to use a fixed API key. +// +// This is the equivalent of providing a headers provider that sets the +// "Authorization" header with "Bearer " + the given key. This will overwrite +// any "Authorization" header that may be on the context or from existing header +// provider. +// +// Note, this uses a fixed header value for authentication. Many users that want +// to rotate this value without reconnecting should use +// [NewAPIKeyDynamicCredentials]. +func NewAPIKeyStaticCredentials(apiKey string) Credentials { + return internal.NewAPIKeyStaticCredentials(apiKey) +} + +// NewAPIKeyDynamicCredentials creates credentials powered by a callback that +// is invoked on each request. The callback accepts the context that is given by +// the calling user and can return a key or an error. When error is non-nil, the +// client call is failed with that error. When string is non-empty, it is used +// as the API key. When string is empty, nothing is set/overridden. +// +// This is the equivalent of providing a headers provider that returns the +// "Authorization" header with "Bearer " + the given function result. If the +// resulting string is non-empty, it will overwrite any "Authorization" header +// that may be on the context or from existing header provider. +func NewAPIKeyDynamicCredentials(apiKeyCallback func(context.Context) (string, error)) Credentials { + return internal.NewAPIKeyDynamicCredentials(apiKeyCallback) +} + +// NewMTLSCredentials creates credentials that use TLS with the client +// certificate as the given one. If the client options do not already enable +// TLS, this enables it. If the client options' TLS configuration is present and +// already has a client certificate, client creation will fail when applying +// these credentials. +func NewMTLSCredentials(certificate tls.Certificate) Credentials { + return internal.NewMTLSCredentials(certificate) +} + +// NewWorkflowUpdateServiceTimeoutOrCanceledError creates a new WorkflowUpdateServiceTimeoutOrCanceledError. +func NewWorkflowUpdateServiceTimeoutOrCanceledError(err error) *WorkflowUpdateServiceTimeoutOrCanceledError { + return internal.NewWorkflowUpdateServiceTimeoutOrCanceledError(err) +} diff --git a/vendor/go.temporal.io/sdk/client/service_proxy.go b/vendor/go.temporal.io/sdk/client/service_proxy.go new file mode 100644 index 00000000000..c9250001cc2 --- /dev/null +++ b/vendor/go.temporal.io/sdk/client/service_proxy.go @@ -0,0 +1,52 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package client + +import ( + "go.temporal.io/api/proxy" + "go.temporal.io/api/workflowservice/v1" +) + +// WorkflowServiceProxyOptions provides options for configuring a WorkflowServiceProxyServer. +// Client is a WorkflowServiceClient used to forward requests received by the server to the +// Temporal Frontend. +type WorkflowServiceProxyOptions struct { + Client workflowservice.WorkflowServiceClient + // DisableHeaderForwarding disables the forwarding of headers from the incoming request to the outgoing request. + DisableHeaderForwarding bool +} + +// NewWorkflowServiceProxyServer creates a WorkflowServiceServer suitable for registering with a GRPC Server. Requests will +// be forwarded to the passed in WorkflowService Client. GRPC interceptors can be added on the Server or Client to adjust +// requests and responses. +func NewWorkflowServiceProxyServer(options WorkflowServiceProxyOptions) (workflowservice.WorkflowServiceServer, error) { + return proxy.NewWorkflowServiceProxyServer(proxy.WorkflowServiceProxyOptions{ + // These options are expected to be kept mostly in sync, but we can't do a + // naive type conversion because we want users to be able to update to newer + // API library versions with older SDK versions. + Client: options.Client, + DisableHeaderForwarding: options.DisableHeaderForwarding, + }) +} diff --git a/vendor/go.temporal.io/sdk/converter/byte_slice_payload_converter.go b/vendor/go.temporal.io/sdk/converter/byte_slice_payload_converter.go new file mode 100644 index 00000000000..36285763748 --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/byte_slice_payload_converter.go @@ -0,0 +1,85 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import ( + "encoding/base64" + "fmt" + "reflect" + + commonpb "go.temporal.io/api/common/v1" +) + +// ByteSlicePayloadConverter pass through []byte to Data field in payload. +type ByteSlicePayloadConverter struct { +} + +// NewByteSlicePayloadConverter creates new instance of ByteSlicePayloadConverter. +func NewByteSlicePayloadConverter() *ByteSlicePayloadConverter { + return &ByteSlicePayloadConverter{} +} + +// ToPayload converts single []byte value to payload. +func (c *ByteSlicePayloadConverter) ToPayload(value interface{}) (*commonpb.Payload, error) { + if valueBytes, isByteSlice := value.([]byte); isByteSlice { + return newPayload(valueBytes, c), nil + } + + return nil, nil +} + +// FromPayload converts single []byte value from payload. +func (c *ByteSlicePayloadConverter) FromPayload(payload *commonpb.Payload, valuePtr interface{}) error { + rv := reflect.ValueOf(valuePtr) + if rv.Kind() != reflect.Ptr || rv.IsNil() { + return fmt.Errorf("type: %T: %w", valuePtr, ErrValuePtrIsNotPointer) + } + v := rv.Elem() + value := payload.Data + if v.Kind() == reflect.Interface { + v.Set(reflect.ValueOf(value)) + } else if v.Kind() == reflect.Slice && v.Type().Elem().Kind() == reflect.Uint8 { + // Must be a []byte. + v.SetBytes(value) + } else { + return fmt.Errorf("type %T: %w", valuePtr, ErrTypeIsNotByteSlice) + } + return nil +} + +// ToString converts payload object into human readable string. +func (c *ByteSlicePayloadConverter) ToString(payload *commonpb.Payload) string { + var byteSlice []byte + err := c.FromPayload(payload, &byteSlice) + if err != nil { + return err.Error() + } + return base64.RawStdEncoding.EncodeToString(byteSlice) +} + +// Encoding returns MetadataEncodingBinary. +func (c *ByteSlicePayloadConverter) Encoding() string { + return MetadataEncodingBinary +} diff --git a/vendor/go.temporal.io/sdk/converter/codec.go b/vendor/go.temporal.io/sdk/converter/codec.go new file mode 100644 index 00000000000..35152306eb1 --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/codec.go @@ -0,0 +1,521 @@ +// The MIT License +// +// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import ( + "bytes" + "compress/zlib" + "encoding/json" + "fmt" + "io" + "net/http" + "strings" + + commonpb "go.temporal.io/api/common/v1" + "google.golang.org/protobuf/encoding/protojson" + "google.golang.org/protobuf/proto" +) + +// PayloadCodec is an codec that encodes or decodes the given payloads. +// +// For example, NewZlibCodec returns a PayloadCodec that can be used for +// compression. +// These can be used (and even chained) in NewCodecDataConverter. +type PayloadCodec interface { + // Encode optionally encodes the given payloads which are guaranteed to never + // be nil. The parameters must not be mutated. + Encode([]*commonpb.Payload) ([]*commonpb.Payload, error) + + // Decode optionally decodes the given payloads which are guaranteed to never + // be nil. The parameters must not be mutated. + // + // For compatibility reasons, implementers should take care not to decode + // payloads that were not previously encoded. + Decode([]*commonpb.Payload) ([]*commonpb.Payload, error) +} + +// ZlibCodecOptions are options for NewZlibCodec. All fields are optional. +type ZlibCodecOptions struct { + // If true, the zlib codec will encode the contents even if there is no size + // benefit. Otherwise, the zlib codec will only use the encoded value if it + // is smaller. + AlwaysEncode bool +} + +type zlibCodec struct{ options ZlibCodecOptions } + +// NewZlibCodec creates a PayloadCodec for use in NewCodecDataConverter +// to support zlib payload compression. +// +// While this serves as a reasonable example of a compression encoder, callers +// may prefer alternative compression algorithms for lots of small payloads. +func NewZlibCodec(options ZlibCodecOptions) PayloadCodec { return &zlibCodec{options} } + +func (z *zlibCodec) Encode(payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { + result := make([]*commonpb.Payload, len(payloads)) + for i, p := range payloads { + // Marshal and write + b, err := proto.Marshal(p) + if err != nil { + return payloads, err + } + var buf bytes.Buffer + w := zlib.NewWriter(&buf) + _, err = w.Write(b) + if closeErr := w.Close(); closeErr != nil && err == nil { + err = closeErr + } + if err != nil { + return payloads, err + } + // Only set if smaller than original amount or has option to always encode + if buf.Len() < len(b) || z.options.AlwaysEncode { + result[i] = &commonpb.Payload{ + Metadata: map[string][]byte{MetadataEncoding: []byte("binary/zlib")}, + Data: buf.Bytes(), + } + } else { + result[i] = p + } + } + return result, nil +} + +func (*zlibCodec) Decode(payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { + result := make([]*commonpb.Payload, len(payloads)) + for i, p := range payloads { + // Only if it's our encoding + if string(p.Metadata[MetadataEncoding]) != "binary/zlib" { + result[i] = p + continue + } + r, err := zlib.NewReader(bytes.NewReader(p.Data)) + if err != nil { + return payloads, err + } + // Read all and unmarshal + b, err := io.ReadAll(r) + if closeErr := r.Close(); closeErr != nil && err == nil { + err = closeErr + } + if err != nil { + return payloads, err + } + result[i] = &commonpb.Payload{} + err = proto.Unmarshal(b, result[i]) + if err != nil { + return payloads, err + } + } + return result, nil +} + +// CodecDataConverter is a DataConverter that wraps an underlying data +// converter and supports chained encoding of just the payload without regard +// for serialization to/from actual types. +type CodecDataConverter struct { + parent DataConverter + codecs []PayloadCodec +} + +// NewCodecDataConverter wraps the given parent DataConverter and performs +// encoding/decoding on the payload via the given codecs. When encoding for +// ToPayload(s), the codecs are applied last to first meaning the earlier +// encoders wrap the later ones. When decoding for FromPayload(s) and +// ToString(s), the decoders are applied first to last to reverse the effect. +func NewCodecDataConverter(parent DataConverter, codecs ...PayloadCodec) DataConverter { + return &CodecDataConverter{parent, codecs} +} + +func (e *CodecDataConverter) encode(payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { + var err error + // Iterate backwards encoding + for i := len(e.codecs) - 1; i >= 0; i-- { + if payloads, err = e.codecs[i].Encode(payloads); err != nil { + return payloads, err + } + } + return payloads, nil +} + +func (e *CodecDataConverter) decode(payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { + var err error + // Iterate forwards decoding + for _, codec := range e.codecs { + if payloads, err = codec.Decode(payloads); err != nil { + return payloads, err + } + } + return payloads, nil +} + +// ToPayload implements DataConverter.ToPayload performing encoding on the +// result of the parent's ToPayload call. +func (e *CodecDataConverter) ToPayload(value interface{}) (*commonpb.Payload, error) { + payload, err := e.parent.ToPayload(value) + if payload == nil || err != nil { + return payload, err + } + + encodedPayloads, err := e.encode([]*commonpb.Payload{payload}) + if err != nil { + return payload, nil + } + if len(encodedPayloads) != 1 { + return payload, fmt.Errorf("received %d payloads from codec, expected 1", len(encodedPayloads)) + } + return encodedPayloads[0], err +} + +// ToPayloads implements DataConverter.ToPayloads performing encoding on the +// result of the parent's ToPayloads call. +func (e *CodecDataConverter) ToPayloads(value ...interface{}) (*commonpb.Payloads, error) { + payloads, err := e.parent.ToPayloads(value...) + if payloads == nil || err != nil { + return payloads, err + } + encodedPayloads, err := e.encode(payloads.Payloads) + return &commonpb.Payloads{Payloads: encodedPayloads}, err +} + +// FromPayload implements DataConverter.FromPayload performing decoding on the +// given payload before sending to the parent FromPayload. +func (e *CodecDataConverter) FromPayload(payload *commonpb.Payload, valuePtr interface{}) error { + if payload == nil { + return nil + } + decodedPayloads, err := e.decode([]*commonpb.Payload{payload}) + if err != nil { + return err + } + if len(decodedPayloads) != 1 { + return fmt.Errorf("received %d payloads from codec, expected 1", len(decodedPayloads)) + } + + return e.parent.FromPayload(decodedPayloads[0], valuePtr) +} + +// FromPayloads implements DataConverter.FromPayloads performing decoding on the +// given payloads before sending to the parent FromPayloads. +func (e *CodecDataConverter) FromPayloads(payloads *commonpb.Payloads, valuePtrs ...interface{}) error { + if payloads == nil { + return e.parent.FromPayloads(payloads, valuePtrs...) + } + decodedPayloads, err := e.decode(payloads.Payloads) + if err != nil { + return err + } + return e.parent.FromPayloads(&commonpb.Payloads{Payloads: decodedPayloads}, valuePtrs...) +} + +// ToString implements DataConverter.ToString performing decoding on the given +// payload before sending to the parent ToString. +func (e *CodecDataConverter) ToString(payload *commonpb.Payload) string { + decodedPayloads, err := e.decode([]*commonpb.Payload{payload}) + if err != nil { + return err.Error() + } + if len(decodedPayloads) != 1 { + return fmt.Errorf("received %d payloads from codec, expected 1", len(decodedPayloads)).Error() + } + return e.parent.ToString(decodedPayloads[0]) +} + +// ToStrings implements DataConverter.ToStrings using ToString for each value. +func (e *CodecDataConverter) ToStrings(payloads *commonpb.Payloads) []string { + if payloads == nil { + return nil + } + strs := make([]string, len(payloads.Payloads)) + // Perform decoding one by one here so that we return individual errors + for i, payload := range payloads.Payloads { + strs[i] = e.ToString(payload) + } + return strs +} + +const remotePayloadCodecEncodePath = "/encode" +const remotePayloadCodecDecodePath = "/decode" + +type codecHTTPHandler struct { + codecs []PayloadCodec +} + +func (e *codecHTTPHandler) encode(payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { + var err error + for i := len(e.codecs) - 1; i >= 0; i-- { + if payloads, err = e.codecs[i].Encode(payloads); err != nil { + return payloads, err + } + } + return payloads, nil +} + +func (e *codecHTTPHandler) decode(payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { + var err error + for _, codec := range e.codecs { + if payloads, err = codec.Decode(payloads); err != nil { + return payloads, err + } + } + return payloads, nil +} + +// ServeHTTP implements the http.Handler interface. +func (e *codecHTTPHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { + if r.Method != "POST" { + http.NotFound(w, r) + return + } + + path := r.URL.Path + + if !strings.HasSuffix(path, remotePayloadCodecEncodePath) && + !strings.HasSuffix(path, remotePayloadCodecDecodePath) { + http.NotFound(w, r) + return + } + + var payloadspb commonpb.Payloads + var err error + + if r.Body == nil { + http.Error(w, http.StatusText(http.StatusBadRequest), http.StatusBadRequest) + return + } + + bs, err := io.ReadAll(r.Body) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + if err = protojson.Unmarshal(bs, &payloadspb); err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + payloads := payloadspb.Payloads + + switch { + case strings.HasSuffix(path, remotePayloadCodecEncodePath): + if payloads, err = e.encode(payloads); err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + case strings.HasSuffix(path, remotePayloadCodecDecodePath): + if payloads, err = e.decode(payloads); err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + default: + http.NotFound(w, r) + return + } + + w.Header().Set("Content-Type", "application/json") + err = json.NewEncoder(w).Encode(commonpb.Payloads{Payloads: payloads}) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } +} + +// NewPayloadCodecHTTPHandler creates a http.Handler for a PayloadCodec. +// This can be used to provide a remote data converter. +func NewPayloadCodecHTTPHandler(e ...PayloadCodec) http.Handler { + return &codecHTTPHandler{codecs: e} +} + +// RemotePayloadCodecOptions are options for RemotePayloadCodec. +// Client is optional. +type RemotePayloadCodecOptions struct { + Endpoint string + ModifyRequest func(*http.Request) error + Client http.Client +} + +type remotePayloadCodec struct { + options RemotePayloadCodecOptions +} + +// NewRemotePayloadCodec creates a PayloadCodec using the remote endpoint configured by RemotePayloadCodecOptions. +func NewRemotePayloadCodec(options RemotePayloadCodecOptions) PayloadCodec { + return &remotePayloadCodec{options} +} + +// Encode uses the remote payload codec endpoint to encode payloads. +func (pc *remotePayloadCodec) Encode(payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { + return pc.encodeOrDecode(pc.options.Endpoint+remotePayloadCodecEncodePath, payloads) +} + +// Decode uses the remote payload codec endpoint to decode payloads. +func (pc *remotePayloadCodec) Decode(payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { + return pc.encodeOrDecode(pc.options.Endpoint+remotePayloadCodecDecodePath, payloads) +} + +func (pc *remotePayloadCodec) encodeOrDecode(endpoint string, payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { + requestPayloads, err := json.Marshal(commonpb.Payloads{Payloads: payloads}) + if err != nil { + return payloads, fmt.Errorf("unable to marshal payloads: %w", err) + } + + req, err := http.NewRequest(http.MethodPost, endpoint, bytes.NewReader(requestPayloads)) + if err != nil { + return payloads, fmt.Errorf("unable to build request: %w", err) + } + + req.Header.Set("Content-Type", "application/json") + + if pc.options.ModifyRequest != nil { + err = pc.options.ModifyRequest(req) + if err != nil { + return payloads, err + } + } + + response, err := pc.options.Client.Do(req) + if err != nil { + return payloads, err + } + defer func() { _ = response.Body.Close() }() + + if response.StatusCode == 200 { + bs, err := io.ReadAll(response.Body) + if err != nil { + return payloads, fmt.Errorf("failed to read response body: %w", err) + } + var resultPayloads commonpb.Payloads + err = protojson.Unmarshal(bs, &resultPayloads) + if err != nil { + return payloads, fmt.Errorf("unable to unmarshal payloads: %w", err) + } + if len(payloads) != len(resultPayloads.Payloads) { + return payloads, fmt.Errorf("received %d payloads from remote codec, expected %d", len(resultPayloads.Payloads), len(payloads)) + } + return resultPayloads.Payloads, nil + } + + message, _ := io.ReadAll(response.Body) + return payloads, fmt.Errorf("%s: %s", http.StatusText(response.StatusCode), message) +} + +// Fields Endpoint, ModifyRequest, Client of RemotePayloadCodecOptions are also +// exposed here in RemoteDataConverterOptions for backwards compatibility. + +// RemoteDataConverterOptions are options for NewRemoteDataConverter. +type RemoteDataConverterOptions struct { + Endpoint string + ModifyRequest func(*http.Request) error + Client http.Client +} + +type remoteDataConverter struct { + parent DataConverter + payloadCodec PayloadCodec +} + +// NewRemoteDataConverter wraps the given parent DataConverter and performs +// encoding/decoding on the payload via the remote endpoint. +func NewRemoteDataConverter(parent DataConverter, options RemoteDataConverterOptions) DataConverter { + options.Endpoint = strings.TrimSuffix(options.Endpoint, "/") + payloadCodec := NewRemotePayloadCodec(RemotePayloadCodecOptions(options)) + return &remoteDataConverter{parent, payloadCodec} +} + +// ToPayload implements DataConverter.ToPayload performing remote encoding on the +// result of the parent's ToPayload call. +func (rdc *remoteDataConverter) ToPayload(value interface{}) (*commonpb.Payload, error) { + payload, err := rdc.parent.ToPayload(value) + if payload == nil || err != nil { + return payload, err + } + encodedPayloads, err := rdc.payloadCodec.Encode([]*commonpb.Payload{payload}) + if err != nil { + return payload, err + } + return encodedPayloads[0], err +} + +// ToPayloads implements DataConverter.ToPayloads performing remote encoding on the +// result of the parent's ToPayloads call. +func (rdc *remoteDataConverter) ToPayloads(value ...interface{}) (*commonpb.Payloads, error) { + payloads, err := rdc.parent.ToPayloads(value...) + if payloads == nil || err != nil { + return payloads, err + } + encodedPayloads, err := rdc.payloadCodec.Encode(payloads.Payloads) + return &commonpb.Payloads{Payloads: encodedPayloads}, err +} + +// FromPayload implements DataConverter.FromPayload performing remote decoding on the +// given payload before sending to the parent FromPayload. +func (rdc *remoteDataConverter) FromPayload(payload *commonpb.Payload, valuePtr interface{}) error { + decodedPayloads, err := rdc.payloadCodec.Decode([]*commonpb.Payload{payload}) + if err != nil { + return err + } + return rdc.parent.FromPayload(decodedPayloads[0], valuePtr) +} + +// FromPayloads implements DataConverter.FromPayloads performing remote decoding on the +// given payloads before sending to the parent FromPayloads. +func (rdc *remoteDataConverter) FromPayloads(payloads *commonpb.Payloads, valuePtrs ...interface{}) error { + if payloads == nil { + return rdc.parent.FromPayloads(payloads, valuePtrs...) + } + + decodedPayloads, err := rdc.payloadCodec.Decode(payloads.Payloads) + if err != nil { + return err + } + return rdc.parent.FromPayloads(&commonpb.Payloads{Payloads: decodedPayloads}, valuePtrs...) +} + +// ToString implements DataConverter.ToString performing remote decoding on the given +// payload before sending to the parent ToString. +func (rdc *remoteDataConverter) ToString(payload *commonpb.Payload) string { + if payload == nil { + return rdc.parent.ToString(payload) + } + + decodedPayloads, err := rdc.payloadCodec.Decode([]*commonpb.Payload{payload}) + if err != nil { + return err.Error() + } + return rdc.parent.ToString(decodedPayloads[0]) +} + +// ToStrings implements DataConverter.ToStrings using ToString for each value. +func (rdc *remoteDataConverter) ToStrings(payloads *commonpb.Payloads) []string { + if payloads == nil { + return nil + } + + strs := make([]string, len(payloads.Payloads)) + // Perform decoding one by one here so that we return individual errors + for i, payload := range payloads.Payloads { + strs[i] = rdc.ToString(payload) + } + return strs +} diff --git a/vendor/go.temporal.io/sdk/converter/composite_data_converter.go b/vendor/go.temporal.io/sdk/converter/composite_data_converter.go new file mode 100644 index 00000000000..822c41a99af --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/composite_data_converter.go @@ -0,0 +1,177 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import ( + "fmt" + + commonpb "go.temporal.io/api/common/v1" +) + +type ( + // CompositeDataConverter applies PayloadConverters in specified order. + CompositeDataConverter struct { + payloadConverters map[string]PayloadConverter + orderedEncodings []string + } +) + +// NewCompositeDataConverter creates a new instance of CompositeDataConverter from an ordered list of PayloadConverters. +// Order is important here because during serialization the DataConverter will try the PayloadConverters in +// that order until a PayloadConverter returns non nil payload. +// The last PayloadConverter should always serialize the value (JSONPayloadConverter is a good candidate for it). +func NewCompositeDataConverter(payloadConverters ...PayloadConverter) DataConverter { + dc := &CompositeDataConverter{ + payloadConverters: make(map[string]PayloadConverter, len(payloadConverters)), + orderedEncodings: make([]string, len(payloadConverters)), + } + + for i, payloadConverter := range payloadConverters { + dc.payloadConverters[payloadConverter.Encoding()] = payloadConverter + dc.orderedEncodings[i] = payloadConverter.Encoding() + } + + return dc +} + +// ToPayloads converts a list of values. +func (dc *CompositeDataConverter) ToPayloads(values ...interface{}) (*commonpb.Payloads, error) { + if len(values) == 0 { + return nil, nil + } + + result := &commonpb.Payloads{} + for i, value := range values { + payload, err := dc.ToPayload(value) + if err != nil { + return nil, fmt.Errorf("values[%d]: %w", i, err) + } + + result.Payloads = append(result.Payloads, payload) + } + + return result, nil +} + +// FromPayloads converts to a list of values of different types. +func (dc *CompositeDataConverter) FromPayloads(payloads *commonpb.Payloads, valuePtrs ...interface{}) error { + if payloads == nil { + return nil + } + + for i, payload := range payloads.GetPayloads() { + if i >= len(valuePtrs) { + break + } + + err := dc.FromPayload(payload, valuePtrs[i]) + if err != nil { + return fmt.Errorf("payload item %d: %w", i, err) + } + } + + return nil +} + +// ToPayload converts single value to payload. +func (dc *CompositeDataConverter) ToPayload(value interface{}) (*commonpb.Payload, error) { + for _, enc := range dc.orderedEncodings { + payloadConverter := dc.payloadConverters[enc] + payload, err := payloadConverter.ToPayload(value) + if err != nil { + return nil, err + } + if payload != nil { + return payload, nil + } + } + + return nil, fmt.Errorf("value: %v of type: %T: %w", value, value, ErrUnableToFindConverter) +} + +// FromPayload converts single value from payload. +func (dc *CompositeDataConverter) FromPayload(payload *commonpb.Payload, valuePtr interface{}) error { + if payload == nil { + return nil + } + + enc, err := encoding(payload) + if err != nil { + return err + } + + payloadConverter, ok := dc.payloadConverters[enc] + if !ok { + return fmt.Errorf("encoding %s: %w", enc, ErrEncodingIsNotSupported) + } + + return payloadConverter.FromPayload(payload, valuePtr) +} + +// ToString converts payload object into human readable string. +func (dc *CompositeDataConverter) ToString(payload *commonpb.Payload) string { + if payload == nil { + return "" + } + + enc, err := encoding(payload) + if err != nil { + return err.Error() + } + + payloadConverter, ok := dc.payloadConverters[enc] + if !ok { + return fmt.Errorf("encoding %s: %w", enc, ErrEncodingIsNotSupported).Error() + } + + return payloadConverter.ToString(payload) +} + +// ToStrings converts payloads object into human readable strings. +func (dc *CompositeDataConverter) ToStrings(payloads *commonpb.Payloads) []string { + if payloads == nil { + return nil + } + + var result []string + for _, payload := range payloads.GetPayloads() { + result = append(result, dc.ToString(payload)) + } + + return result +} + +func encoding(payload *commonpb.Payload) (string, error) { + metadata := payload.GetMetadata() + if metadata == nil { + return "", ErrMetadataIsNotSet + } + + if e, ok := metadata[MetadataEncoding]; ok { + return string(e), nil + } + + return "", ErrEncodingIsNotSet +} diff --git a/vendor/go.temporal.io/sdk/converter/data_converter.go b/vendor/go.temporal.io/sdk/converter/data_converter.go new file mode 100644 index 00000000000..d4019679594 --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/data_converter.go @@ -0,0 +1,66 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import ( + commonpb "go.temporal.io/api/common/v1" +) + +type ( + // DataConverter is used by the framework to serialize/deserialize input and output of activity/workflow + // that need to be sent over the wire. + // To encode/decode workflow arguments, set DataConverter in client, through client.Options. + // To override DataConverter for specific activity or child workflow use workflow.WithDataConverter to create new Context, + // and pass that context to ExecuteActivity/ExecuteChildWorkflow calls. + // Temporal support using different DataConverters for different activity/childWorkflow in same workflow. + // For advanced data converters that may exceed the deadlock detection timeout + // for a workflow, such as ones making remote calls, use + // workflow.DataConverterWithoutDeadlockDetection. + DataConverter interface { + // ToPayload converts single value to payload. + ToPayload(value interface{}) (*commonpb.Payload, error) + // FromPayload converts single value from payload. + // + // Note, values should not be reused for extraction here because merging on + // top of existing values may result in unexpected behavior similar to + // json.Unmarshal. + FromPayload(payload *commonpb.Payload, valuePtr interface{}) error + + // ToPayloads converts a list of values. + ToPayloads(value ...interface{}) (*commonpb.Payloads, error) + // FromPayloads converts to a list of values of different types. + // Useful for deserializing arguments of function invocations. + // + // Note, values should not be reused for extraction here because merging on + // top of existing values may result in unexpected behavior similar to + // json.Unmarshal. + FromPayloads(payloads *commonpb.Payloads, valuePtrs ...interface{}) error + + // ToString converts payload object into human readable string. + ToString(input *commonpb.Payload) string + // ToStrings converts payloads object into human readable strings. + ToStrings(input *commonpb.Payloads) []string + } +) diff --git a/vendor/go.temporal.io/sdk/converter/default_data_converter.go b/vendor/go.temporal.io/sdk/converter/default_data_converter.go new file mode 100644 index 00000000000..0b8d9a06a1c --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/default_data_converter.go @@ -0,0 +1,46 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +var ( + defaultDataConverter = NewCompositeDataConverter( + NewNilPayloadConverter(), + NewByteSlicePayloadConverter(), + + // Order is important here. Both ProtoJsonPayload and ProtoPayload converters check for the same proto.Message + // interface. The first match (ProtoJsonPayload in this case) will always be used for serialization. + // Deserialization is controlled by metadata, therefore both converters can deserialize corresponding data format + // (JSON or binary proto). + NewProtoJSONPayloadConverter(), + NewProtoPayloadConverter(), + + NewJSONPayloadConverter(), + ) +) + +// GetDefaultDataConverter returns default data converter used by Temporal worker. +func GetDefaultDataConverter() DataConverter { + return defaultDataConverter +} diff --git a/vendor/go.temporal.io/sdk/converter/errors.go b/vendor/go.temporal.io/sdk/converter/errors.go new file mode 100644 index 00000000000..40ef49cea4b --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/errors.go @@ -0,0 +1,54 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import ( + "errors" +) + +var ( + // ErrMetadataIsNotSet is returned when metadata is not set. + ErrMetadataIsNotSet = errors.New("metadata is not set") + // ErrEncodingIsNotSet is returned when payload encoding metadata is not set. + ErrEncodingIsNotSet = errors.New("payload encoding metadata is not set") + // ErrEncodingIsNotSupported is returned when payload encoding is not supported. + ErrEncodingIsNotSupported = errors.New("payload encoding is not supported") + // ErrUnableToEncode is returned when unable to encode. + ErrUnableToEncode = errors.New("unable to encode") + // ErrUnableToDecode is returned when unable to decode. + ErrUnableToDecode = errors.New("unable to decode") + // ErrUnableToSetValue is returned when unable to set value. + ErrUnableToSetValue = errors.New("unable to set value") + // ErrUnableToFindConverter is returned when unable to find converter. + ErrUnableToFindConverter = errors.New("unable to find converter") + // ErrTypeNotImplementProtoMessage is returned when value doesn't implement proto.Message. + ErrTypeNotImplementProtoMessage = errors.New("type doesn't implement proto.Message") + // ErrValuePtrIsNotPointer is returned when proto value is not a pointer. + ErrValuePtrIsNotPointer = errors.New("not a pointer type") + // ErrValuePtrMustConcreteType is returned when proto value is of interface type. + ErrValuePtrMustConcreteType = errors.New("must be a concrete type, not interface") + // ErrTypeIsNotByteSlice is returned when value is not of *[]byte type. + ErrTypeIsNotByteSlice = errors.New("type is not *[]byte") +) diff --git a/vendor/go.temporal.io/sdk/converter/failure_converter.go b/vendor/go.temporal.io/sdk/converter/failure_converter.go new file mode 100644 index 00000000000..3bd1e7001c7 --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/failure_converter.go @@ -0,0 +1,67 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import failurepb "go.temporal.io/api/failure/v1" + +// FailureConverter is used by the sdk to serialize/deserialize errors +// that need to be sent over the wire. +// To use a custom FailureConverter, set FailureConverter in client, through client.Options. +type FailureConverter interface { + // ErrorToFailure converts a error to a Failure proto message. + ErrorToFailure(err error) *failurepb.Failure + + // FailureToError converts a Failure proto message to a Go Error. + FailureToError(failure *failurepb.Failure) error +} + +type encodedFailure struct { + Message string `json:"message"` + StackTrace string `json:"stack_trace"` +} + +// EncodeCommonFailureAttributes packs failure attributes to a payload so that they flow through a dataconverter. +func EncodeCommonFailureAttributes(dc DataConverter, failure *failurepb.Failure) error { + var err error + + failure.EncodedAttributes, err = dc.ToPayload(encodedFailure{ + Message: failure.Message, + StackTrace: failure.StackTrace, + }) + if err != nil { + return err + } + failure.Message = "Encoded failure" + failure.StackTrace = "" + + return nil +} + +// DecodeCommonFailureAttributes unpacks failure attributes from a stored payload, if present. +func DecodeCommonFailureAttributes(dc DataConverter, failure *failurepb.Failure) { + var ea encodedFailure + if failure.GetEncodedAttributes() != nil && dc.FromPayload(failure.GetEncodedAttributes(), &ea) == nil { + failure.Message = ea.Message + failure.StackTrace = ea.StackTrace + } +} diff --git a/vendor/go.temporal.io/sdk/converter/grpc_interceptor.go b/vendor/go.temporal.io/sdk/converter/grpc_interceptor.go new file mode 100644 index 00000000000..0e30fbe1e41 --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/grpc_interceptor.go @@ -0,0 +1,114 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import ( + "fmt" + + "google.golang.org/grpc" + + commonpb "go.temporal.io/api/common/v1" + failurepb "go.temporal.io/api/failure/v1" + "go.temporal.io/api/proxy" +) + +// PayloadCodecGRPCClientInterceptorOptions holds interceptor options. +// Currently this is just the list of codecs to use. +type PayloadCodecGRPCClientInterceptorOptions struct { + Codecs []PayloadCodec +} + +// NewPayloadCodecGRPCClientInterceptor returns a GRPC Client Interceptor that will mimic the encoding +// that the SDK system would perform when configured with a matching EncodingDataConverter. +// Note: This approach does not support use cases that rely on the ContextAware DataConverter interface as +// workflow context is not available at the GRPC level. +func NewPayloadCodecGRPCClientInterceptor(options PayloadCodecGRPCClientInterceptorOptions) (grpc.UnaryClientInterceptor, error) { + return proxy.NewPayloadVisitorInterceptor(proxy.PayloadVisitorInterceptorOptions{ + Outbound: &proxy.VisitPayloadsOptions{ + Visitor: func(vpc *proxy.VisitPayloadsContext, payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { + var err error + for i := len(options.Codecs) - 1; i >= 0; i-- { + if payloads, err = options.Codecs[i].Encode(payloads); err != nil { + return payloads, err + } + } + + return payloads, nil + }, + SkipSearchAttributes: true, + }, + Inbound: &proxy.VisitPayloadsOptions{ + Visitor: func(vpc *proxy.VisitPayloadsContext, payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { + var err error + for _, codec := range options.Codecs { + if payloads, err = codec.Decode(payloads); err != nil { + return payloads, err + } + } + + return payloads, nil + }, + SkipSearchAttributes: true, + }, + }) +} + +// NewFailureGRPCClientInterceptorOptions holds interceptor options. +type NewFailureGRPCClientInterceptorOptions struct { + // DataConverter is optional. If not set the SDK's dataconverter will be used. + DataConverter DataConverter + // Whether to Encode attributes. The current implementation requires this be true. + EncodeCommonAttributes bool +} + +// NewFailureGRPCClientInterceptor returns a GRPC Client Interceptor that will mimic the encoding +// that the SDK system would perform when configured with a FailureConverter with the EncodeCommonAttributes option set. +// When combining this with NewPayloadCodecGRPCClientInterceptor you should ensure that NewFailureGRPCClientInterceptor is +// before NewPayloadCodecGRPCClientInterceptor in the chain. +func NewFailureGRPCClientInterceptor(options NewFailureGRPCClientInterceptorOptions) (grpc.UnaryClientInterceptor, error) { + if !options.EncodeCommonAttributes { + return nil, fmt.Errorf("EncodeCommonAttributes must be set for this interceptor to function") + } + + dc := options.DataConverter + if dc == nil { + dc = GetDefaultDataConverter() + } + + return proxy.NewFailureVisitorInterceptor(proxy.FailureVisitorInterceptorOptions{ + Outbound: &proxy.VisitFailuresOptions{ + Visitor: func(vpc *proxy.VisitFailuresContext, failure *failurepb.Failure) error { + return EncodeCommonFailureAttributes(dc, failure) + }, + }, + Inbound: &proxy.VisitFailuresOptions{ + Visitor: func(vpc *proxy.VisitFailuresContext, failure *failurepb.Failure) error { + DecodeCommonFailureAttributes(dc, failure) + + return nil + }, + }, + }) +} diff --git a/vendor/go.temporal.io/sdk/converter/json_payload_converter.go b/vendor/go.temporal.io/sdk/converter/json_payload_converter.go new file mode 100644 index 00000000000..6d25dc6ec12 --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/json_payload_converter.go @@ -0,0 +1,69 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import ( + "encoding/json" + "fmt" + + commonpb "go.temporal.io/api/common/v1" +) + +// JSONPayloadConverter converts to/from JSON. +type JSONPayloadConverter struct { +} + +// NewJSONPayloadConverter creates a new instance of JSONPayloadConverter. +func NewJSONPayloadConverter() *JSONPayloadConverter { + return &JSONPayloadConverter{} +} + +// ToPayload converts a single value to a payload. +func (c *JSONPayloadConverter) ToPayload(value interface{}) (*commonpb.Payload, error) { + data, err := json.Marshal(value) + if err != nil { + return nil, fmt.Errorf("%w: %v", ErrUnableToEncode, err) + } + return newPayload(data, c), nil +} + +// FromPayload converts a single payload to a value. +func (c *JSONPayloadConverter) FromPayload(payload *commonpb.Payload, valuePtr interface{}) error { + err := json.Unmarshal(payload.GetData(), valuePtr) + if err != nil { + return fmt.Errorf("%w: %v", ErrUnableToDecode, err) + } + return nil +} + +// ToString converts a payload object into a human-readable string. +func (c *JSONPayloadConverter) ToString(payload *commonpb.Payload) string { + return string(payload.GetData()) +} + +// Encoding returns MetadataEncodingJSON. +func (c *JSONPayloadConverter) Encoding() string { + return MetadataEncodingJSON +} diff --git a/vendor/go.temporal.io/sdk/converter/metadata.go b/vendor/go.temporal.io/sdk/converter/metadata.go new file mode 100644 index 00000000000..84b3cee062d --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/metadata.go @@ -0,0 +1,42 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +const ( + // MetadataEncoding is "encoding" + MetadataEncoding = "encoding" + // MetadataMessageType is "messageType" + MetadataMessageType = "messageType" + // MetadataEncodingBinary is "binary/plain" + MetadataEncodingBinary = "binary/plain" + // MetadataEncodingJSON is "json/plain" + MetadataEncodingJSON = "json/plain" + // MetadataEncodingNil is "binary/null" + MetadataEncodingNil = "binary/null" + // MetadataEncodingProtoJSON is "json/protobuf" + MetadataEncodingProtoJSON = "json/protobuf" + // MetadataEncodingProto is "binary/protobuf" + MetadataEncodingProto = "binary/protobuf" +) diff --git a/vendor/go.temporal.io/sdk/converter/nil_payload_converter.go b/vendor/go.temporal.io/sdk/converter/nil_payload_converter.go new file mode 100644 index 00000000000..9eca1a009fd --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/nil_payload_converter.go @@ -0,0 +1,75 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import ( + "fmt" + "reflect" + + commonpb "go.temporal.io/api/common/v1" +) + +// NilPayloadConverter doesn't set Data field in payload. +type NilPayloadConverter struct { +} + +// NewNilPayloadConverter creates new instance of NilPayloadConverter. +func NewNilPayloadConverter() *NilPayloadConverter { + return &NilPayloadConverter{} +} + +// ToPayload converts single nil value to payload. +func (c *NilPayloadConverter) ToPayload(value interface{}) (*commonpb.Payload, error) { + if isInterfaceNil(value) { + return newPayload(nil, c), nil + } + return nil, nil +} + +// FromPayload converts single nil value from payload. +func (c *NilPayloadConverter) FromPayload(_ *commonpb.Payload, valuePtr interface{}) error { + originalValue := reflect.ValueOf(valuePtr) + if originalValue.Kind() != reflect.Ptr { + return fmt.Errorf("type: %T: %w", valuePtr, ErrValuePtrIsNotPointer) + } + + originalValue = originalValue.Elem() + if !originalValue.CanSet() { + return fmt.Errorf("type: %T: %w", valuePtr, ErrUnableToSetValue) + } + + originalValue.Set(reflect.Zero(originalValue.Type())) + return nil +} + +// ToString converts payload object into human readable string. +func (c *NilPayloadConverter) ToString(*commonpb.Payload) string { + return "nil" +} + +// Encoding returns MetadataEncodingNil. +func (c *NilPayloadConverter) Encoding() string { + return MetadataEncodingNil +} diff --git a/vendor/go.temporal.io/sdk/converter/payload_converter.go b/vendor/go.temporal.io/sdk/converter/payload_converter.go new file mode 100644 index 00000000000..6858abb1f0d --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/payload_converter.go @@ -0,0 +1,72 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import ( + commonpb "go.temporal.io/api/common/v1" +) + +// PayloadConverter is an interface to convert a single payload. +type PayloadConverter interface { + // ToPayload converts a single value to payload. It should return nil if the + // PayloadConverter can not convert the passed value (i.e. type is unknown). + ToPayload(value interface{}) (*commonpb.Payload, error) + // FromPayload converts single value from payload. valuePtr should be a + // reference to a variable of a type corresponding to the payload + // encoding. Otherwise it should return error. + FromPayload(payload *commonpb.Payload, valuePtr interface{}) error + // ToString converts payload object into human readable string. + ToString(*commonpb.Payload) string + + // Encoding returns encoding supported by PayloadConverter. + Encoding() string +} + +type protoPayloadConverterInterface interface { + PayloadConverter + ExcludeProtobufMessageTypes() bool +} + +func newPayload(data []byte, c PayloadConverter) *commonpb.Payload { + return &commonpb.Payload{ + Metadata: map[string][]byte{ + MetadataEncoding: []byte(c.Encoding()), + }, + Data: data, + } +} + +func newProtoPayload(data []byte, c protoPayloadConverterInterface, messageType string) *commonpb.Payload { + if !c.ExcludeProtobufMessageTypes() { + return &commonpb.Payload{ + Metadata: map[string][]byte{ + MetadataEncoding: []byte(c.Encoding()), + MetadataMessageType: []byte(messageType), + }, + Data: data, + } + } + return newPayload(data, c) +} diff --git a/vendor/go.temporal.io/sdk/converter/proto_json_payload_converter.go b/vendor/go.temporal.io/sdk/converter/proto_json_payload_converter.go new file mode 100644 index 00000000000..e8eb563837c --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/proto_json_payload_converter.go @@ -0,0 +1,238 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import ( + "bytes" + "encoding/json" + "fmt" + "reflect" + + gogojsonpb "github.com/gogo/protobuf/jsonpb" + gogoproto "github.com/gogo/protobuf/proto" + commonpb "go.temporal.io/api/common/v1" + "go.temporal.io/api/temporalproto" + "google.golang.org/protobuf/encoding/protojson" + "google.golang.org/protobuf/proto" +) + +// ProtoJSONPayloadConverter converts proto objects to/from JSON. +type ProtoJSONPayloadConverter struct { + gogoMarshaler gogojsonpb.Marshaler + gogoUnmarshaler gogojsonpb.Unmarshaler + protoMarshalOptions protojson.MarshalOptions + protoUnmarshalOptions protojson.UnmarshalOptions + temporalProtoUnmarshalOptions temporalproto.CustomJSONUnmarshalOptions + options ProtoJSONPayloadConverterOptions +} + +// ProtoJSONPayloadConverterOptions represents options for `NewProtoJSONPayloadConverterWithOptions`. +type ProtoJSONPayloadConverterOptions struct { + // ExcludeProtobufMessageTypes prevents the message type (`my.package.MyMessage`) + // from being included in the Payload. + ExcludeProtobufMessageTypes bool + + // AllowUnknownFields will ignore unknown fields when unmarshalling, as opposed to returning an error + AllowUnknownFields bool + + // UseProtoNames uses proto field name instead of lowerCamelCase name in JSON + // field names. + UseProtoNames bool + + // UseEnumNumbers emits enum values as numbers. + UseEnumNumbers bool + + // EmitUnpopulated specifies whether to emit unpopulated fields. + EmitUnpopulated bool + + // LegacyTemporalProtoCompat will allow enums serialized as SCREAMING_SNAKE_CASE. + // Useful for backwards compatibility when migrating a proto message from gogoproto to standard protobuf. + LegacyTemporalProtoCompat bool +} + +var ( + jsonNil, _ = json.Marshal(nil) +) + +// NewProtoJSONPayloadConverter creates new instance of `ProtoJSONPayloadConverter`. +func NewProtoJSONPayloadConverter() *ProtoJSONPayloadConverter { + return &ProtoJSONPayloadConverter{ + gogoMarshaler: gogojsonpb.Marshaler{}, + gogoUnmarshaler: gogojsonpb.Unmarshaler{}, + protoMarshalOptions: protojson.MarshalOptions{}, + protoUnmarshalOptions: protojson.UnmarshalOptions{}, + temporalProtoUnmarshalOptions: temporalproto.CustomJSONUnmarshalOptions{}, + } +} + +// NewProtoJSONPayloadConverterWithOptions creates new instance of `ProtoJSONPayloadConverter` with the provided options. +func NewProtoJSONPayloadConverterWithOptions(options ProtoJSONPayloadConverterOptions) *ProtoJSONPayloadConverter { + return &ProtoJSONPayloadConverter{ + gogoMarshaler: gogojsonpb.Marshaler{ + EnumsAsInts: options.UseEnumNumbers, + EmitDefaults: options.EmitUnpopulated, + OrigName: options.UseProtoNames, + }, + gogoUnmarshaler: gogojsonpb.Unmarshaler{ + AllowUnknownFields: options.AllowUnknownFields, + }, + protoMarshalOptions: protojson.MarshalOptions{ + UseProtoNames: options.UseProtoNames, + UseEnumNumbers: options.UseEnumNumbers, + EmitUnpopulated: options.EmitUnpopulated, + }, + protoUnmarshalOptions: protojson.UnmarshalOptions{ + DiscardUnknown: options.AllowUnknownFields, + }, + temporalProtoUnmarshalOptions: temporalproto.CustomJSONUnmarshalOptions{ + DiscardUnknown: options.AllowUnknownFields, + }, + options: options, + } +} + +// ToPayload converts single proto value to payload. +func (c *ProtoJSONPayloadConverter) ToPayload(value interface{}) (*commonpb.Payload, error) { + // Proto golang structs might be generated with 4 different protoc plugin versions: + // 1. github.com/golang/protobuf - ~v1.3.5 is the most recent pre-APIv2 version of APIv1. + // 2. github.com/golang/protobuf - ^v1.4.0 is a version of APIv1 implemented in terms of APIv2. + // 3. google.golang.org/protobuf - ^v1.20.0 is APIv2. + // 4. github.com/gogo/protobuf - any version. + // Case 1 is not supported. + // Cases 2 and 3 implements proto.Message and are the same in this context. + // Case 4 implements gogoproto.Message. + // It is important to check for proto.Message first because cases 2 and 3 also implement gogoproto.Message. + + if isInterfaceNil(value) { + return newPayload(jsonNil, c), nil + } + + builtPointer := false + for { + if valueProto, ok := value.(proto.Message); ok { + byteSlice, err := c.protoMarshalOptions.Marshal(valueProto) + if err != nil { + return nil, fmt.Errorf("%w: %v", ErrUnableToEncode, err) + } + return newProtoPayload(byteSlice, c, string(valueProto.ProtoReflect().Descriptor().FullName())), nil + } + if valueGogoProto, ok := value.(gogoproto.Message); ok { + var buf bytes.Buffer + err := c.gogoMarshaler.Marshal(&buf, valueGogoProto) + if err != nil { + return nil, fmt.Errorf("%w: %v", ErrUnableToEncode, err) + } + return newProtoPayload(buf.Bytes(), c, gogoproto.MessageName(valueGogoProto)), nil + } + if builtPointer { + break + } + value = pointerTo(value).Interface() + builtPointer = true + } + + return nil, nil +} + +// FromPayload converts single proto value from payload. +func (c *ProtoJSONPayloadConverter) FromPayload(payload *commonpb.Payload, valuePtr interface{}) error { + originalValue := reflect.ValueOf(valuePtr) + if originalValue.Kind() != reflect.Ptr { + return fmt.Errorf("type: %T: %w", valuePtr, ErrValuePtrIsNotPointer) + } + + originalValue = originalValue.Elem() + if !originalValue.CanSet() { + return fmt.Errorf("type: %T: %w", valuePtr, ErrUnableToSetValue) + } + + if bytes.Equal(payload.GetData(), jsonNil) { + originalValue.Set(reflect.Zero(originalValue.Type())) + return nil + } + + if originalValue.Kind() == reflect.Interface { + return fmt.Errorf("value type: %s: %w", originalValue.Type().String(), ErrValuePtrMustConcreteType) + } + + value := originalValue + // If original value is of value type (i.e. commonpb.WorkflowType), create a pointer to it. + if originalValue.Kind() != reflect.Ptr { + value = pointerTo(originalValue.Interface()) + } + + protoValue := value.Interface() // protoValue is for sure of pointer type (i.e. *commonpb.WorkflowType). + gogoProtoMessage, isGogoProtoMessage := protoValue.(gogoproto.Message) + protoMessage, isProtoMessage := protoValue.(proto.Message) + if !isGogoProtoMessage && !isProtoMessage { + return fmt.Errorf("type: %T: %w", protoValue, ErrTypeNotImplementProtoMessage) + } + + // If original value is nil, create new instance. + if originalValue.Kind() == reflect.Ptr && originalValue.IsNil() { + value = newOfSameType(originalValue) + protoValue = value.Interface() + if isProtoMessage { + protoMessage = protoValue.(proto.Message) // type assertion must always succeed + } else if isGogoProtoMessage { + gogoProtoMessage = protoValue.(gogoproto.Message) // type assertion must always succeed + } + } + + var err error + if isProtoMessage { + if c.options.LegacyTemporalProtoCompat { + err = c.temporalProtoUnmarshalOptions.Unmarshal(payload.GetData(), protoMessage) + } else { + err = c.protoUnmarshalOptions.Unmarshal(payload.GetData(), protoMessage) + } + } else if isGogoProtoMessage { + err = c.gogoUnmarshaler.Unmarshal(bytes.NewReader(payload.GetData()), gogoProtoMessage) + } + // If original value wasn't a pointer then set value back to where valuePtr points to. + if originalValue.Kind() != reflect.Ptr { + originalValue.Set(value.Elem()) + } + + if err != nil { + return fmt.Errorf("%w: %v", ErrUnableToDecode, err) + } + + return nil +} + +// ToString converts payload object into human readable string. +func (c *ProtoJSONPayloadConverter) ToString(payload *commonpb.Payload) string { + return string(payload.GetData()) +} + +// Encoding returns MetadataEncodingProtoJSON. +func (c *ProtoJSONPayloadConverter) Encoding() string { + return MetadataEncodingProtoJSON +} + +func (c *ProtoJSONPayloadConverter) ExcludeProtobufMessageTypes() bool { + return c.options.ExcludeProtobufMessageTypes +} diff --git a/vendor/go.temporal.io/sdk/converter/proto_payload_converter.go b/vendor/go.temporal.io/sdk/converter/proto_payload_converter.go new file mode 100644 index 00000000000..507ab3ba9d4 --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/proto_payload_converter.go @@ -0,0 +1,170 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import ( + "encoding/base64" + "fmt" + "reflect" + + gogoproto "github.com/gogo/protobuf/proto" + commonpb "go.temporal.io/api/common/v1" + "google.golang.org/protobuf/proto" +) + +// ProtoPayloadConverter converts proto objects to protobuf binary format. +type ProtoPayloadConverter struct { + options ProtoPayloadConverterOptions +} + +// ProtoPayloadConverterOptions represents options for `NewProtoPayloadConverterWithOptions`. +type ProtoPayloadConverterOptions struct { + // ExcludeProtobufMessageTypes prevents the message type (`my.package.MyMessage`) + // from being included in the Payload. + ExcludeProtobufMessageTypes bool +} + +// NewProtoPayloadConverter creates new instance of `ProtoPayloadConverter“. +func NewProtoPayloadConverter() *ProtoPayloadConverter { + return &ProtoPayloadConverter{} +} + +// NewProtoPayloadConverterWithOptions creates new instance of `ProtoPayloadConverter` with the provided options. +func NewProtoPayloadConverterWithOptions(options ProtoPayloadConverterOptions) *ProtoPayloadConverter { + return &ProtoPayloadConverter{ + options: options, + } +} + +// ToPayload converts single proto value to payload. +func (c *ProtoPayloadConverter) ToPayload(value interface{}) (*commonpb.Payload, error) { + // Proto golang structs might be generated with 4 different protoc plugin versions: + // 1. github.com/golang/protobuf - ~v1.3.5 is the most recent pre-APIv2 version of APIv1. + // 2. github.com/golang/protobuf - ^v1.4.0 is a version of APIv1 implemented in terms of APIv2. + // 3. google.golang.org/protobuf - ^v1.20.0 is APIv2. + // 4. github.com/gogo/protobuf - any version. + // Case 1 is not supported. + // Cases 2 and 3 implements proto.Message and are the same in this context. + // Case 4 implements gogoproto.Message. + // It is important to check for proto.Message first because cases 2 and 3 also implements gogoproto.Message. + + builtPointer := false + for { + if valueProto, ok := value.(proto.Message); ok { + byteSlice, err := proto.Marshal(valueProto) + if err != nil { + return nil, fmt.Errorf("%w: %v", ErrUnableToEncode, err) + } + return newProtoPayload(byteSlice, c, string(valueProto.ProtoReflect().Descriptor().FullName())), nil + } + if valueGogoProto, ok := value.(gogoproto.Message); ok { + data, err := gogoproto.Marshal(valueGogoProto) + if err != nil { + return nil, fmt.Errorf("%w: %v", ErrUnableToEncode, err) + } + return newProtoPayload(data, c, gogoproto.MessageName(valueGogoProto)), nil + } + if builtPointer { + break + } + value = pointerTo(value).Interface() + builtPointer = true + } + + return nil, nil +} + +// FromPayload converts single proto value from payload. +func (c *ProtoPayloadConverter) FromPayload(payload *commonpb.Payload, valuePtr interface{}) error { + originalValue := reflect.ValueOf(valuePtr) + if originalValue.Kind() != reflect.Ptr { + return fmt.Errorf("type: %T: %w", valuePtr, ErrValuePtrIsNotPointer) + } + + originalValue = originalValue.Elem() + if !originalValue.CanSet() { + return fmt.Errorf("type: %T: %w", valuePtr, ErrUnableToSetValue) + } + + if originalValue.Kind() == reflect.Interface { + return fmt.Errorf("value type: %s: %w", originalValue.Type().String(), ErrValuePtrMustConcreteType) + } + + value := originalValue + // If original value is of value type (i.e. commonpb.WorkflowType), create a pointer to it. + if originalValue.Kind() != reflect.Ptr { + value = pointerTo(originalValue.Interface()) + } + + protoValue := value.Interface() // protoValue is for sure of pointer type (i.e. *commonpb.WorkflowType). + gogoProtoMessage, isGogoProtoMessage := protoValue.(gogoproto.Message) + protoMessage, isProtoMessage := protoValue.(proto.Message) + if !isGogoProtoMessage && !isProtoMessage { + return fmt.Errorf("type: %T: %w", protoValue, ErrTypeNotImplementProtoMessage) + } + + // If original value is nil, create new instance. + if originalValue.Kind() == reflect.Ptr && originalValue.IsNil() { + value = newOfSameType(originalValue) + protoValue = value.Interface() + if isProtoMessage { + protoMessage = protoValue.(proto.Message) // type assertion must always succeed + } else if isGogoProtoMessage { + gogoProtoMessage = protoValue.(gogoproto.Message) // type assertion must always succeed + } + } + + var err error + if isProtoMessage { + err = proto.Unmarshal(payload.GetData(), protoMessage) + } else if isGogoProtoMessage { + err = gogoproto.Unmarshal(payload.GetData(), gogoProtoMessage) + } + // If original value wasn't a pointer then set value back to where valuePtr points to. + if originalValue.Kind() != reflect.Ptr { + originalValue.Set(value.Elem()) + } + + if err != nil { + return fmt.Errorf("%w: %v", ErrUnableToDecode, err) + } + + return nil +} + +// ToString converts payload object into human readable string. +func (c *ProtoPayloadConverter) ToString(payload *commonpb.Payload) string { + // We can't do anything better here. + return base64.RawStdEncoding.EncodeToString(payload.GetData()) +} + +// Encoding returns MetadataEncodingProto. +func (c *ProtoPayloadConverter) Encoding() string { + return MetadataEncodingProto +} + +func (c *ProtoPayloadConverter) ExcludeProtobufMessageTypes() bool { + return c.options.ExcludeProtobufMessageTypes +} diff --git a/vendor/go.temporal.io/sdk/converter/reflect.go b/vendor/go.temporal.io/sdk/converter/reflect.go new file mode 100644 index 00000000000..4da46dab089 --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/reflect.go @@ -0,0 +1,47 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import ( + "reflect" +) + +func pointerTo(val interface{}) reflect.Value { + valPtr := reflect.New(reflect.TypeOf(val)) + valPtr.Elem().Set(reflect.ValueOf(val)) + return valPtr +} + +func newOfSameType(val reflect.Value) reflect.Value { + valType := val.Type().Elem() // is value type (i.e. commonpb.WorkflowType) + newValue := reflect.New(valType) // is of pointer type (i.e. *commonpb.WorkflowType) + val.Set(newValue) // set newly created value back to passed value + return newValue +} + +func isInterfaceNil(i interface{}) bool { + v := reflect.ValueOf(i) + return i == nil || (v.Kind() == reflect.Ptr && v.IsNil()) +} diff --git a/vendor/go.temporal.io/sdk/converter/value.go b/vendor/go.temporal.io/sdk/converter/value.go new file mode 100644 index 00000000000..a750d43765b --- /dev/null +++ b/vendor/go.temporal.io/sdk/converter/value.go @@ -0,0 +1,51 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +type ( + // EncodedValue is used to encapsulate/extract encoded value from workflow/activity. + EncodedValue interface { + // HasValue return whether there is value encoded. + HasValue() bool + // Get extract the encoded value into strong typed value pointer. + // + // Note, values should not be reused for extraction here because merging on + // top of existing values may result in unexpected behavior similar to + // json.Unmarshal. + Get(valuePtr interface{}) error + } + + // EncodedValues is used to encapsulate/extract encoded one or more values from workflow/activity. + EncodedValues interface { + // HasValues return whether there are values encoded. + HasValues() bool + // Get extract the encoded values into strong typed value pointers. + // + // Note, values should not be reused for extraction here because merging on + // top of existing values may result in unexpected behavior similar to + // json.Unmarshal. + Get(valuePtr ...interface{}) error + } +) diff --git a/vendor/go.temporal.io/sdk/internal/activity.go b/vendor/go.temporal.io/sdk/internal/activity.go new file mode 100644 index 00000000000..d115b1181b9 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/activity.go @@ -0,0 +1,393 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "fmt" + "time" + + commonpb "go.temporal.io/api/common/v1" + "go.temporal.io/api/workflowservice/v1" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/log" +) + +type ( + // ActivityType identifies an activity type. + ActivityType struct { + Name string + } + + // ActivityInfo contains information about a currently executing activity. + ActivityInfo struct { + TaskToken []byte + WorkflowType *WorkflowType + WorkflowNamespace string + WorkflowExecution WorkflowExecution + ActivityID string + ActivityType ActivityType + TaskQueue string + HeartbeatTimeout time.Duration // Maximum time between heartbeats. 0 means no heartbeat needed. + ScheduledTime time.Time // Time of activity scheduled by a workflow + StartedTime time.Time // Time of activity start + Deadline time.Time // Time of activity timeout + Attempt int32 // Attempt starts from 1, and increased by 1 for every retry if retry policy is specified. + IsLocalActivity bool // true if it is a local activity + } + + // RegisterActivityOptions consists of options for registering an activity. + RegisterActivityOptions struct { + // When an activity is a function the name is an actual activity type name. + // When an activity is part of a structure then each member of the structure becomes an activity with + // this Name as a prefix + activity function name. + // + // If this is set, users are strongly recommended to set + // worker.Options.DisableRegistrationAliasing at the worker level to prevent + // ambiguity between string names and function references. Also users should + // always use this string name when executing this activity. + Name string + DisableAlreadyRegisteredCheck bool + + // When registering a struct with activities, skip functions that are not valid activities. If false, + // registration panics. + SkipInvalidStructFunctions bool + } + + // ActivityOptions stores all activity-specific parameters that will be stored inside of a context. + // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is + // subjected to change in the future. + ActivityOptions struct { + // TaskQueue - Name of the task queue that the activity needs to be scheduled on. + // Optional: The default task queue with the same name as the workflow task queue. + TaskQueue string + + // ScheduleToCloseTimeout - Total time that a workflow is willing to wait for an Activity to complete. + // ScheduleToCloseTimeout limits the total time of an Activity's execution including retries + // (use StartToCloseTimeout to limit the time of a single attempt). + // The zero value of this uses default value. + // Either this option or StartToCloseTimeout is required: Defaults to unlimited. + ScheduleToCloseTimeout time.Duration + + // ScheduleToStartTimeout - Time that the Activity Task can stay in the Task Queue before it is picked up by + // a Worker. Do not specify this timeout unless using host specific Task Queues for Activity Tasks are being + // used for routing. In almost all situations that don't involve routing activities to specific hosts, it is + // better to rely on the default value. + // ScheduleToStartTimeout is always non-retryable. Retrying after this timeout doesn't make sense, as it would + // just put the Activity Task back into the same Task Queue. + // Optional: Defaults to unlimited. + ScheduleToStartTimeout time.Duration + + // StartToCloseTimeout - Maximum time of a single Activity execution attempt. + // Note that the Temporal Server doesn't detect Worker process failures directly. It relies on this timeout + // to detect that an Activity that didn't complete on time. So this timeout should be as short as the longest + // possible execution of the Activity body. Potentially long running Activities must specify HeartbeatTimeout + // and call Activity.RecordHeartbeat(ctx, "my-heartbeat") periodically for timely failure detection. + // Either this option or ScheduleToCloseTimeout is required: Defaults to the ScheduleToCloseTimeout value. + StartToCloseTimeout time.Duration + + // HeartbeatTimeout - Heartbeat interval. Activity must call Activity.RecordHeartbeat(ctx, "my-heartbeat") + // before this interval passes after the last heartbeat or the Activity starts. + HeartbeatTimeout time.Duration + + // WaitForCancellation - Whether to wait for canceled activity to be completed( + // activity can be failed, completed, cancel accepted) + // Optional: default false + WaitForCancellation bool + + // ActivityID - Business level activity ID, this is not needed for most of the cases if you have + // to specify this then talk to the temporal team. This is something will be done in the future. + // Optional: default empty string + ActivityID string + + // RetryPolicy - Specifies how to retry an Activity if an error occurs. + // More details are available at docs.temporal.io. + // RetryPolicy is optional. If one is not specified, a default RetryPolicy is provided by the server. + // The default RetryPolicy provided by the server specifies: + // - InitialInterval of 1 second + // - BackoffCoefficient of 2.0 + // - MaximumInterval of 100 x InitialInterval + // - MaximumAttempts of 0 (unlimited) + // To disable retries, set MaximumAttempts to 1. + // The default RetryPolicy provided by the server can be overridden by the dynamic config. + RetryPolicy *RetryPolicy + + // If true, eager execution will not be requested, regardless of worker settings. + // If false, eager execution may still be disabled at the worker level or + // may not be requested due to lack of available slots. + // + // Eager activity execution means the server returns requested eager + // activities directly from the workflow task back to this worker. This is + // faster than non-eager, which may be dispatched to a separate worker. + DisableEagerExecution bool + + // VersioningIntent - Specifies whether this activity should run on a worker with a compatible + // build ID or not. See temporal.VersioningIntent. + // WARNING: Worker versioning is currently experimental + VersioningIntent VersioningIntent + } + + // LocalActivityOptions stores local activity specific parameters that will be stored inside of a context. + LocalActivityOptions struct { + // ScheduleToCloseTimeout - The end to end timeout for the local activity, including retries. + // At least one of ScheduleToCloseTimeout or StartToCloseTimeout is required. + // Defaults to StartToCloseTimeout if not set. + ScheduleToCloseTimeout time.Duration + + // StartToCloseTimeout - The timeout for a single execution of the local activity. + // At least one of ScheduleToCloseTimeout or StartToCloseTimeout is required. + // Defaults to ScheduleToCloseTimeout if not set. + StartToCloseTimeout time.Duration + + // RetryPolicy - Specify how to retry activity if error happens. + // Optional: default is to retry according to the default retry policy up to ScheduleToCloseTimeout + // with 1sec initial delay between retries and 2x backoff. + RetryPolicy *RetryPolicy + } +) + +// GetActivityInfo returns information about the currently executing activity. +func GetActivityInfo(ctx context.Context) ActivityInfo { + return getActivityOutboundInterceptor(ctx).GetInfo(ctx) +} + +// HasHeartbeatDetails checks if there are heartbeat details from last attempt. +func HasHeartbeatDetails(ctx context.Context) bool { + return getActivityOutboundInterceptor(ctx).HasHeartbeatDetails(ctx) +} + +// IsActivity checks if the context is an activity context from a normal or local activity. +func IsActivity(ctx context.Context) bool { + a := ctx.Value(activityInterceptorContextKey) + return a != nil +} + +// GetHeartbeatDetails extracts heartbeat details from the last failed attempt. This is used in combination with the retry policy. +// An activity could be scheduled with an optional retry policy on ActivityOptions. If the activity failed, then server +// would attempt to dispatch another activity task to retry according to the retry policy. If there were heartbeat +// details reported by activity from the failed attempt, the details would be delivered along with the activity task for +// the retry attempt. An activity can extract the details from GetHeartbeatDetails() and resume progress from there. +// +// Note: Values should not be reused for extraction here because merging on top +// of existing values may result in unexpected behavior similar to json.Unmarshal. +func GetHeartbeatDetails(ctx context.Context, d ...interface{}) error { + return getActivityOutboundInterceptor(ctx).GetHeartbeatDetails(ctx, d...) +} + +// GetActivityLogger returns a logger that can be used in the activity. +func GetActivityLogger(ctx context.Context) log.Logger { + return getActivityOutboundInterceptor(ctx).GetLogger(ctx) +} + +// GetActivityMetricsHandler returns a metrics handler that can be used in the activity. +func GetActivityMetricsHandler(ctx context.Context) metrics.Handler { + return getActivityOutboundInterceptor(ctx).GetMetricsHandler(ctx) +} + +// GetWorkerStopChannel returns a read-only channel. The closure of this channel indicates the activity worker is stopping. +// When the worker is stopping, it will close this channel and wait until the worker stop timeout finishes. After the timeout +// hits, the worker will cancel the activity context and then exit. The timeout can be defined by worker option: WorkerStopTimeout. +// Use this channel to handle a graceful activity exit when the activity worker stops. +func GetWorkerStopChannel(ctx context.Context) <-chan struct{} { + return getActivityOutboundInterceptor(ctx).GetWorkerStopChannel(ctx) +} + +// RecordActivityHeartbeat sends a heartbeat for the currently executing activity. +// If the activity is either canceled or workflow/activity doesn't exist, then we would cancel +// the context with error context.Canceled. +// +// TODO: we don't have a way to distinguish between the two cases when context is canceled because +// context doesn't support overriding value of ctx.Error. +// TODO: Implement automatic heartbeating with cancellation through ctx. +// +// details - The details that you provided here can be seen in the workflow when it receives TimeoutError. You +// can check error TimeoutType()/Details(). +func RecordActivityHeartbeat(ctx context.Context, details ...interface{}) { + getActivityOutboundInterceptor(ctx).RecordHeartbeat(ctx, details...) +} + +// ServiceInvoker abstracts calls to the Temporal service from an activity implementation. +// Implement to unit test activities. +type ServiceInvoker interface { + // Returns ActivityTaskCanceledError if activity is canceled + Heartbeat(ctx context.Context, details *commonpb.Payloads, skipBatching bool) error + Close(ctx context.Context, flushBufferedHeartbeat bool) + GetClient(options ClientOptions) Client +} + +// WithActivityTask adds activity specific information into context. +// Use this method to unit test activity implementations that use context extractor methodshared. +func WithActivityTask( + ctx context.Context, + task *workflowservice.PollActivityTaskQueueResponse, + taskQueue string, + invoker ServiceInvoker, + logger log.Logger, + metricsHandler metrics.Handler, + dataConverter converter.DataConverter, + workerStopChannel <-chan struct{}, + contextPropagators []ContextPropagator, + interceptors []WorkerInterceptor, +) (context.Context, error) { + scheduled := task.GetScheduledTime().AsTime() + started := task.GetStartedTime().AsTime() + scheduleToCloseTimeout := task.GetScheduleToCloseTimeout().AsDuration() + startToCloseTimeout := task.GetStartToCloseTimeout().AsDuration() + heartbeatTimeout := task.GetHeartbeatTimeout().AsDuration() + deadline := calculateActivityDeadline(scheduled, started, scheduleToCloseTimeout, startToCloseTimeout) + + logger = log.With(logger, + tagActivityID, task.ActivityId, + tagActivityType, task.ActivityType.GetName(), + tagAttempt, task.Attempt, + tagWorkflowType, task.WorkflowType.GetName(), + tagWorkflowID, task.WorkflowExecution.WorkflowId, + tagRunID, task.WorkflowExecution.RunId, + ) + + return newActivityContext(ctx, interceptors, &activityEnvironment{ + taskToken: task.TaskToken, + serviceInvoker: invoker, + activityType: ActivityType{Name: task.ActivityType.GetName()}, + activityID: task.ActivityId, + workflowExecution: WorkflowExecution{ + RunID: task.WorkflowExecution.RunId, + ID: task.WorkflowExecution.WorkflowId}, + logger: logger, + metricsHandler: metricsHandler, + deadline: deadline, + heartbeatTimeout: heartbeatTimeout, + scheduledTime: scheduled, + startedTime: started, + taskQueue: taskQueue, + dataConverter: dataConverter, + attempt: task.GetAttempt(), + heartbeatDetails: task.HeartbeatDetails, + workflowType: &WorkflowType{ + Name: task.WorkflowType.GetName(), + }, + workflowNamespace: task.WorkflowNamespace, + workerStopChannel: workerStopChannel, + contextPropagators: contextPropagators, + }) +} + +// WithLocalActivityTask adds local activity specific information into context. +func WithLocalActivityTask( + ctx context.Context, + task *localActivityTask, + logger log.Logger, + metricsHandler metrics.Handler, + dataConverter converter.DataConverter, + interceptors []WorkerInterceptor, +) (context.Context, error) { + if ctx == nil { + ctx = context.Background() + } + workflowTypeLocal := task.params.WorkflowInfo.WorkflowType + workflowType := task.params.WorkflowInfo.WorkflowType.Name + activityType := task.params.ActivityType + logger = log.With(logger, + tagActivityID, task.activityID, + tagActivityType, activityType, + tagAttempt, task.attempt, + tagWorkflowType, workflowType, + tagWorkflowID, task.params.WorkflowInfo.WorkflowExecution.ID, + tagRunID, task.params.WorkflowInfo.WorkflowExecution.RunID, + ) + startedTime := time.Now() + scheduleToCloseTimeout := task.params.ScheduleToCloseTimeout + startToCloseTimeout := task.params.StartToCloseTimeout + + if startToCloseTimeout == 0 { + startToCloseTimeout = scheduleToCloseTimeout + } + if scheduleToCloseTimeout == 0 { + scheduleToCloseTimeout = startToCloseTimeout + } + deadline := calculateActivityDeadline(task.scheduledTime, startedTime, scheduleToCloseTimeout, startToCloseTimeout) + if task.attempt > 1 && !task.expireTime.IsZero() && task.expireTime.Before(deadline) { + // this is attempt and expire time is before SCHEDULE_TO_CLOSE timeout + deadline = task.expireTime + } + return newActivityContext(ctx, interceptors, &activityEnvironment{ + workflowType: &workflowTypeLocal, + workflowNamespace: task.params.WorkflowInfo.Namespace, + taskQueue: task.params.WorkflowInfo.TaskQueueName, + activityType: ActivityType{Name: activityType}, + activityID: fmt.Sprintf("%v", task.activityID), + workflowExecution: task.params.WorkflowInfo.WorkflowExecution, + logger: logger, + metricsHandler: metricsHandler, + isLocalActivity: true, + deadline: deadline, + scheduledTime: task.scheduledTime, + startedTime: startedTime, + dataConverter: dataConverter, + attempt: task.attempt, + }) +} + +func newActivityContext( + ctx context.Context, + interceptors []WorkerInterceptor, + env *activityEnvironment, +) (context.Context, error) { + ctx = context.WithValue(ctx, activityEnvContextKey, env) + + // Create interceptor with default inbound and outbound values and put on + // context + envInterceptor := &activityEnvironmentInterceptor{env: env} + envInterceptor.inboundInterceptor = envInterceptor + envInterceptor.outboundInterceptor = envInterceptor + ctx = context.WithValue(ctx, activityEnvInterceptorContextKey, envInterceptor) + ctx = context.WithValue(ctx, activityInterceptorContextKey, envInterceptor.outboundInterceptor) + + // Intercept, run init, and put the new outbound interceptor on the context + for i := len(interceptors) - 1; i >= 0; i-- { + envInterceptor.inboundInterceptor = interceptors[i].InterceptActivity(ctx, envInterceptor.inboundInterceptor) + } + err := envInterceptor.inboundInterceptor.Init(envInterceptor) + if err != nil { + return nil, err + } + ctx = context.WithValue(ctx, activityInterceptorContextKey, envInterceptor.outboundInterceptor) + + return ctx, nil +} + +func calculateActivityDeadline(scheduled, started time.Time, scheduleToCloseTimeout, startToCloseTimeout time.Duration) time.Time { + startToCloseDeadline := started.Add(startToCloseTimeout) + if scheduleToCloseTimeout > 0 { + scheduleToCloseDeadline := scheduled.Add(scheduleToCloseTimeout) + // Minimum of the two deadlines. + if scheduleToCloseDeadline.Before(startToCloseDeadline) { + return scheduleToCloseDeadline + } + } + return startToCloseDeadline +} diff --git a/vendor/go.temporal.io/sdk/internal/client.go b/vendor/go.temporal.io/sdk/internal/client.go new file mode 100644 index 00000000000..2e6745caed9 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/client.go @@ -0,0 +1,1247 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "crypto/tls" + "errors" + "fmt" + "sync/atomic" + "time" + + "go.temporal.io/api/cloud/cloudservice/v1" + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/api/operatorservice/v1" + "go.temporal.io/api/workflowservice/v1" + "google.golang.org/grpc" + "google.golang.org/grpc/metadata" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" + ilog "go.temporal.io/sdk/internal/log" + "go.temporal.io/sdk/log" +) + +const ( + // DefaultNamespace is the namespace name which is used if not passed with options. + DefaultNamespace = "default" + + // QueryTypeStackTrace is the build in query type for Client.QueryWorkflow() call. Use this query type to get the call + // stack of the workflow. The result will be a string encoded in the EncodedValue. + QueryTypeStackTrace string = "__stack_trace" + + // QueryTypeOpenSessions is the build in query type for Client.QueryWorkflow() call. Use this query type to get all open + // sessions in the workflow. The result will be a list of SessionInfo encoded in the EncodedValue. + QueryTypeOpenSessions string = "__open_sessions" + + // QueryTypeWorkflowMetadata is the query name for the workflow metadata. + QueryTypeWorkflowMetadata string = "__temporal_workflow_metadata" +) + +type ( + // Client is the client for starting and getting information about a workflow executions as well as + // completing activities asynchronously. + Client interface { + // ExecuteWorkflow starts a workflow execution and return a WorkflowRun instance and error + // The user can use this to start using a function or workflow type name. + // Either by + // ExecuteWorkflow(ctx, options, "workflowTypeName", arg1, arg2, arg3) + // or + // ExecuteWorkflow(ctx, options, workflowExecuteFn, arg1, arg2, arg3) + // The errors it can return: + // - serviceerror.NamespaceNotFound, if namespace does not exist + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // + // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is + // subjected to change in the future. + // + // WorkflowRun has three methods: + // - GetID() string: which return workflow ID (which is same as StartWorkflowOptions.ID if provided) + // - GetRunID() string: which return the first started workflow run ID (please see below) + // - Get(ctx context.Context, valuePtr interface{}) error: which will fill the workflow + // execution result to valuePtr, if workflow execution is a success, or return corresponding + // error. This is a blocking API. + // NOTE: if the started workflow return ContinueAsNewError during the workflow execution, the + // return result of GetRunID() will be the started workflow run ID, not the new run ID caused by ContinueAsNewError, + // however, Get(ctx context.Context, valuePtr interface{}) will return result from the run which did not return ContinueAsNewError. + // Say ExecuteWorkflow started a workflow, in its first run, has run ID "run ID 1", and returned ContinueAsNewError, + // the second run has run ID "run ID 2" and return some result other than ContinueAsNewError: + // GetRunID() will always return "run ID 1" and Get(ctx context.Context, valuePtr interface{}) will return the result of second run. + // NOTE: DO NOT USE THIS API INSIDE A WORKFLOW, USE workflow.ExecuteChildWorkflow instead + ExecuteWorkflow(ctx context.Context, options StartWorkflowOptions, workflow interface{}, args ...interface{}) (WorkflowRun, error) + + // GetWorkflow retrieves a workflow execution and return a WorkflowRun instance + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. + // + // WorkflowRun has three methods: + // - GetID() string: which return workflow ID (which is same as StartWorkflowOptions.ID if provided) + // - GetRunID() string: which return the first started workflow run ID (please see below) + // - Get(ctx context.Context, valuePtr interface{}) error: which will fill the workflow + // execution result to valuePtr, if workflow execution is a success, or return corresponding + // error. This is a blocking API. + // NOTE: if the retrieved workflow returned ContinueAsNewError during the workflow execution, the + // return result of GetRunID() will be the retrieved workflow run ID, not the new run ID caused by ContinueAsNewError, + // however, Get(ctx context.Context, valuePtr interface{}) will return result from the run which did not return ContinueAsNewError. + GetWorkflow(ctx context.Context, workflowID string, runID string) WorkflowRun + + // SignalWorkflow sends a signals to a workflow in execution + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // - signalName name to identify the signal. + // The errors it can return: + // - serviceerror.NotFound + // - serviceerror.Internal + // - serviceerror.Unavailable + SignalWorkflow(ctx context.Context, workflowID string, runID string, signalName string, arg interface{}) error + + // SignalWithStartWorkflow sends a signal to a running workflow. + // If the workflow is not running or not found, it starts the workflow and then sends the signal in transaction. + // - workflowID, signalName, signalArg are same as SignalWorkflow's parameters + // - options, workflow, workflowArgs are same as StartWorkflow's parameters + // - the workflowID parameter is used instead of options.ID. If the latter is present, it must match the workflowID. + // Note: options.WorkflowIDReusePolicy is default to AllowDuplicate. + // The errors it can return: + // - serviceerror.NotFound + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + SignalWithStartWorkflow(ctx context.Context, workflowID string, signalName string, signalArg interface{}, + options StartWorkflowOptions, workflow interface{}, workflowArgs ...interface{}) (WorkflowRun, error) + + // CancelWorkflow cancels a workflow in execution + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // The errors it can return: + // - serviceerror.NotFound + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + CancelWorkflow(ctx context.Context, workflowID string, runID string) error + + // TerminateWorkflow terminates a workflow execution. + // workflowID is required, other parameters are optional. + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // The errors it can return: + // - serviceerror.NotFound + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + TerminateWorkflow(ctx context.Context, workflowID string, runID string, reason string, details ...interface{}) error + + // GetWorkflowHistory gets history events of a particular workflow + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. + // - whether use long poll for tracking new events: when the workflow is running, there can be new events generated during iteration + // of HistoryEventIterator, if isLongPoll == true, then iterator will do long poll, tracking new history event, i.e. the iteration + // will not be finished until workflow is finished; if isLongPoll == false, then iterator will only return current history events. + // - whether return all history events or just the last event, which contains the workflow execution end result + // Example:- + // To iterate all events, + // iter := GetWorkflowHistory(ctx, workflowID, runID, isLongPoll, filterType) + // events := []*shared.HistoryEvent{} + // for iter.HasNext() { + // event, err := iter.Next() + // if err != nil { + // return err + // } + // events = append(events, event) + // } + GetWorkflowHistory(ctx context.Context, workflowID string, runID string, isLongPoll bool, filterType enumspb.HistoryEventFilterType) HistoryEventIterator + + // CompleteActivity reports activity completed. + // activity Execute method can return activity.ErrResultPending to + // indicate the activity is not completed when it's Execute method returns. In that case, this CompleteActivity() method + // should be called when that activity is completed with the actual result and error. If err is nil, activity task + // completed event will be reported; if err is CanceledError, activity task canceled event will be reported; otherwise, + // activity task failed event will be reported. + // An activity implementation should use GetActivityInfo(ctx).TaskToken function to get task token to use for completion. + // Example:- + // To complete with a result. + // CompleteActivity(token, "Done", nil) + // To fail the activity with an error. + // CompleteActivity(token, nil, temporal.NewApplicationError("reason", details) + // The activity can fail with below errors ApplicationError, TimeoutError, CanceledError. + CompleteActivity(ctx context.Context, taskToken []byte, result interface{}, err error) error + + // CompleteActivityByID reports activity completed. + // Similar to CompleteActivity, but may save user from keeping taskToken info. + // activity Execute method can return activity.ErrResultPending to + // indicate the activity is not completed when it's Execute method returns. In that case, this CompleteActivityById() method + // should be called when that activity is completed with the actual result and error. If err is nil, activity task + // completed event will be reported; if err is CanceledError, activity task canceled event will be reported; otherwise, + // activity task failed event will be reported. + // An activity implementation should use activityID provided in ActivityOption to use for completion. + // namespace name, workflowID, activityID are required, runID is optional. + // The errors it can return: + // - ApplicationError + // - TimeoutError + // - CanceledError + CompleteActivityByID(ctx context.Context, namespace, workflowID, runID, activityID string, result interface{}, err error) error + + // RecordActivityHeartbeat records heartbeat for an activity. + // details - is the progress you want to record along with heart beat for this activity. + // The errors it can return: + // - serviceerror.NotFound + // - serviceerror.Internal + // - serviceerror.Unavailable + RecordActivityHeartbeat(ctx context.Context, taskToken []byte, details ...interface{}) error + + // RecordActivityHeartbeatByID records heartbeat for an activity. + // details - is the progress you want to record along with heart beat for this activity. + // The errors it can return: + // - serviceerror.NotFound + // - serviceerror.Internal + // - serviceerror.Unavailable + RecordActivityHeartbeatByID(ctx context.Context, namespace, workflowID, runID, activityID string, details ...interface{}) error + + // ListClosedWorkflow gets closed workflow executions based on request filters + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NamespaceNotFound + ListClosedWorkflow(ctx context.Context, request *workflowservice.ListClosedWorkflowExecutionsRequest) (*workflowservice.ListClosedWorkflowExecutionsResponse, error) + + // ListOpenWorkflow gets open workflow executions based on request filters + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NamespaceNotFound + ListOpenWorkflow(ctx context.Context, request *workflowservice.ListOpenWorkflowExecutionsRequest) (*workflowservice.ListOpenWorkflowExecutionsResponse, error) + + // ListWorkflow gets workflow executions based on query.The query is basically the SQL WHERE clause, + // examples: + // - "(WorkflowID = 'wid1' or (WorkflowType = 'type2' and WorkflowID = 'wid2'))". + // - "CloseTime between '2019-08-27T15:04:05+00:00' and '2019-08-28T15:04:05+00:00'". + // - to list only open workflow use "CloseTime is null" + // Retrieved workflow executions are sorted by StartTime in descending order when list open workflow, + // and sorted by CloseTime in descending order for other queries. + // For supported operations on different server versions see [Visibility]. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // + // [Visibility]: https://docs.temporal.io/visibility + ListWorkflow(ctx context.Context, request *workflowservice.ListWorkflowExecutionsRequest) (*workflowservice.ListWorkflowExecutionsResponse, error) + + // ListArchivedWorkflow gets archived workflow executions based on query. This API will return BadRequest if Temporal + // cluster or target namespace is not configured for visibility archival or read is not enabled. The query is basically the SQL WHERE clause. + // However, different visibility archivers have different limitations on the query. Please check the documentation of the visibility archiver used + // by your namespace to see what kind of queries are accept and whether retrieved workflow executions are ordered or not. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + ListArchivedWorkflow(ctx context.Context, request *workflowservice.ListArchivedWorkflowExecutionsRequest) (*workflowservice.ListArchivedWorkflowExecutionsResponse, error) + + // ScanWorkflow gets workflow executions based on query. The query is basically the SQL WHERE clause + // (see ListWorkflow for query examples). + // ScanWorkflow should be used when retrieving large amount of workflows and order is not needed. + // It will use more resources than ListWorkflow, but will be several times faster + // when retrieving millions of workflows. + // For supported operations on different server versions see [Visibility]. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // [Visibility]: https://docs.temporal.io/visibility + ScanWorkflow(ctx context.Context, request *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error) + + // CountWorkflow gets number of workflow executions based on query. The query is basically the SQL WHERE clause + // (see ListWorkflow for query examples). + // For supported operations on different server versions see [Visibility]. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // + // [Visibility]: https://docs.temporal.io/visibility + CountWorkflow(ctx context.Context, request *workflowservice.CountWorkflowExecutionsRequest) (*workflowservice.CountWorkflowExecutionsResponse, error) + + // GetSearchAttributes returns valid search attributes keys and value types. + // The search attributes can be used in query of List/Scan/Count APIs. Adding new search attributes requires temporal server + // to update dynamic config ValidSearchAttributes. + GetSearchAttributes(ctx context.Context) (*workflowservice.GetSearchAttributesResponse, error) + + // QueryWorkflow queries a given workflow execution and returns the query result synchronously. Parameter workflowID + // and queryType are required, other parameters are optional. The workflowID and runID (optional) identify the + // target workflow execution that this query will be send to. If runID is not specified (empty string), server will + // use the currently running execution of that workflowID. The queryType specifies the type of query you want to + // run. By default, temporal supports "__stack_trace" as a standard query type, which will return string value + // representing the call stack of the target workflow. The target workflow could also setup different query handler + // to handle custom query types. + // See comments at workflow.SetQueryHandler(ctx Context, queryType string, handler interface{}) for more details + // on how to setup query handler within the target workflow. + // - workflowID is required. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // - queryType is the type of the query. + // - args... are the optional query parameters. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NotFound + // - serviceerror.QueryFailed + QueryWorkflow(ctx context.Context, workflowID string, runID string, queryType string, args ...interface{}) (converter.EncodedValue, error) + + // QueryWorkflowWithOptions queries a given workflow execution and returns the query result synchronously. + // See QueryWorkflowWithOptionsRequest and QueryWorkflowWithOptionsResponse for more information. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NotFound + // - serviceerror.QueryFailed + QueryWorkflowWithOptions(ctx context.Context, request *QueryWorkflowWithOptionsRequest) (*QueryWorkflowWithOptionsResponse, error) + + // DescribeWorkflowExecution returns information about the specified workflow execution. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NotFound + DescribeWorkflowExecution(ctx context.Context, workflowID, runID string) (*workflowservice.DescribeWorkflowExecutionResponse, error) + + // DescribeTaskQueue returns information about the target taskqueue, right now this API returns the + // pollers which polled this taskqueue in last few minutes. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NotFound + DescribeTaskQueue(ctx context.Context, taskqueue string, taskqueueType enumspb.TaskQueueType) (*workflowservice.DescribeTaskQueueResponse, error) + + // ResetWorkflowExecution reset an existing workflow execution to WorkflowTaskFinishEventId(exclusive). + // And it will immediately terminating the current execution instance. + // RequestId is used to deduplicate requests. It will be autogenerated if not set. + ResetWorkflowExecution(ctx context.Context, request *workflowservice.ResetWorkflowExecutionRequest) (*workflowservice.ResetWorkflowExecutionResponse, error) + + // UpdateWorkerBuildIdCompatibility allows you to update the worker-build-id based version sets for a particular + // task queue. This is used in conjunction with workers who specify their build id and thus opt into the + // feature. + UpdateWorkerBuildIdCompatibility(ctx context.Context, options *UpdateWorkerBuildIdCompatibilityOptions) error + + // GetWorkerBuildIdCompatibility returns the worker-build-id based version sets for a particular task queue. + GetWorkerBuildIdCompatibility(ctx context.Context, options *GetWorkerBuildIdCompatibilityOptions) (*WorkerBuildIDVersionSets, error) + + // GetWorkerTaskReachability returns which versions are is still in use by open or closed workflows. + GetWorkerTaskReachability(ctx context.Context, options *GetWorkerTaskReachabilityOptions) (*WorkerTaskReachability, error) + + // DescribeTaskQueueEnhanced returns information about the target task queue, broken down by Build Id: + // - List of pollers + // - Workflow Reachability status + // - Backlog info for Workflow and/or Activity tasks + // When not supported by the server, it returns an empty [TaskQueueDescription] if there is no information + // about the task queue, or an error when the response identifies an unsupported server. + // Note that using a sticky queue as target is not supported. + // Also, workflow reachability status is eventually consistent, and it could take a few minutes to update. + // WARNING: Worker versioning is currently experimental, and requires server 1.24+ + DescribeTaskQueueEnhanced(ctx context.Context, options DescribeTaskQueueEnhancedOptions) (TaskQueueDescription, error) + + // UpdateWorkerVersioningRules allows updating the worker-build-id based assignment and redirect rules for a given + // task queue. This is used in conjunction with workers who specify their build id and thus opt into the feature. + // The errors it can return: + // - serviceerror.FailedPrecondition when the conflict token is invalid + // WARNING: Worker versioning is currently experimental, and requires server 1.24+ + UpdateWorkerVersioningRules(ctx context.Context, options UpdateWorkerVersioningRulesOptions) (*WorkerVersioningRules, error) + + // GetWorkerVersioningRules returns the worker-build-id assignment and redirect rules for a task queue. + // WARNING: Worker versioning is currently experimental, and requires server 1.24+ + GetWorkerVersioningRules(ctx context.Context, options GetWorkerVersioningOptions) (*WorkerVersioningRules, error) + + // CheckHealth performs a server health check using the gRPC health check + // API. If the check fails, an error is returned. + CheckHealth(ctx context.Context, request *CheckHealthRequest) (*CheckHealthResponse, error) + + // UpdateWorkflow issues an update request to the + // specified workflow execution and returns a handle to the update that + // is running in in parallel with the calling thread. Errors returned + // from the server will be exposed through the return value of + // WorkflowExecutionUpdateHandle.Get(). Errors that occur before the + // update is requested (e.g. if the required workflow ID field is + // missing from the UpdateWorkflowOptions) are returned + // directly from this function call. + // NOTE: Experimental + UpdateWorkflow(ctx context.Context, options UpdateWorkflowOptions) (WorkflowUpdateHandle, error) + + // GetWorkflowUpdateHandle creates a handle to the referenced update + // which can be polled for an outcome. Note that runID is optional and + // if not specified the most recent runID will be used. + // NOTE: Experimental + GetWorkflowUpdateHandle(GetWorkflowUpdateHandleOptions) WorkflowUpdateHandle + + // WorkflowService provides access to the underlying gRPC service. This should only be used for advanced use cases + // that cannot be accomplished via other Client methods. Unlike calls to other Client methods, calls directly to the + // service are not configured with internal semantics such as automatic retries. + WorkflowService() workflowservice.WorkflowServiceClient + + // OperatorService creates a new operator service client with the same gRPC connection as this client. + OperatorService() operatorservice.OperatorServiceClient + + // Schedule creates a new shedule client with the same gRPC connection as this client. + ScheduleClient() ScheduleClient + + // Close client and clean up underlying resources. + Close() + } + + // ClientOptions are optional parameters for Client creation. + ClientOptions struct { + // Optional: To set the host:port for this client to connect to. + // default: localhost:7233 + // + // This is a gRPC address and therefore can also support a special-formatted address of ":///" that + // will use a registered resolver. By default all hosts returned from the resolver will be used in a round-robin + // fashion. + // + // The "dns" resolver is registered by and used by default. + // + // A custom resolver can be created to provide multiple hosts in other ways. For example, to manually provide + // multiple IPs to round-robin across, a google.golang.org/grpc/resolver/manual resolver can be created and + // registered with google.golang.org/grpc/resolver with a custom scheme: + // builder := manual.NewBuilderWithScheme("myresolver") + // builder.InitialState(resolver.State{Addresses: []resolver.Address{{Addr: "1.2.3.4:1234"}, {Addr: "2.3.4.5:2345"}}}) + // resolver.Register(builder) + // c, err := client.Dial(client.Options{HostPort: "myresolver:///ignoredvalue"}) + // Other more advanced resolvers can also be registered. + HostPort string + + // Optional: To set the namespace name for this client to work with. + // default: default + Namespace string + + // Optional: Set the credentials for this client. + Credentials Credentials + + // Optional: Logger framework can use to log. + // default: default logger provided. + Logger log.Logger + + // Optional: Metrics handler for reporting metrics. + // default: no metrics. + MetricsHandler metrics.Handler + + // Optional: Sets an identify that can be used to track this host for debugging. + // default: default identity that include hostname, groupName and process ID. + Identity string + + // Optional: Sets DataConverter to customize serialization/deserialization of arguments in Temporal + // default: defaultDataConverter, an combination of google protobuf converter, gogo protobuf converter and json converter + DataConverter converter.DataConverter + + // Optional: Sets FailureConverter to customize serialization/deserialization of errors. + // default: temporal.DefaultFailureConverter, does not encode any fields of the error. Use temporal.NewDefaultFailureConverter + // options to configure or create a custom converter. + FailureConverter converter.FailureConverter + + // Optional: Sets ContextPropagators that allows users to control the context information passed through a workflow + // default: nil + ContextPropagators []ContextPropagator + + // Optional: Sets options for server connection that allow users to control features of connections such as TLS settings. + // default: no extra options + ConnectionOptions ConnectionOptions + + // Optional: HeadersProvider will be invoked on every outgoing gRPC request and gives user ability to + // set custom request headers. This can be used to set auth headers for example. + HeadersProvider HeadersProvider + + // Optional parameter that is designed to be used *in tests*. It gets invoked last in + // the gRPC interceptor chain and can be used to induce artificial failures in test scenarios. + TrafficController TrafficController + + // Interceptors to apply to some calls of the client. Earlier interceptors + // wrap later interceptors. + // + // Any interceptors that also implement Interceptor (meaning they implement + // WorkerInterceptor in addition to ClientInterceptor) will be used for + // worker interception as well. When worker interceptors are here and in + // worker options, the ones here wrap the ones in worker options. The same + // interceptor should not be set here and in worker options. + Interceptors []ClientInterceptor + + // If set true, error code labels will not be included on request failure metrics. + DisableErrorCodeMetricTags bool + } + + CloudOperationsClient interface { + CloudService() cloudservice.CloudServiceClient + Close() + } + + // CloudOperationsClientOptions are parameters for CloudOperationsClient creation. + // + // WARNING: Cloud operations client is currently experimental. + CloudOperationsClientOptions struct { + // Optional: The credentials for this client. This is essentially required. + // See [go.temporal.io/sdk/client.NewAPIKeyStaticCredentials], + // [go.temporal.io/sdk/client.NewAPIKeyDynamicCredentials], and + // [go.temporal.io/sdk/client.NewMTLSCredentials]. + // Default: No credentials. + Credentials Credentials + + // Optional: Version header for safer mutations. May or may not be required + // depending on cloud settings. + // Default: No header. + Version string + + // Optional: Advanced server connection options such as TLS settings. Not + // usually needed. + ConnectionOptions ConnectionOptions + + // Optional: Logger framework can use to log. + // Default: Default logger provided. + Logger log.Logger + + // Optional: Metrics handler for reporting metrics. + // Default: No metrics + MetricsHandler metrics.Handler + + // Optional: Overrides the specific host to connect to. Not usually needed. + // Default: saas-api.tmprl.cloud:443 + HostPort string + + // Optional: Disable TLS. + // Default: false (i.e. TLS enabled) + DisableTLS bool + } + + // HeadersProvider returns a map of gRPC headers that should be used on every request. + HeadersProvider interface { + GetHeaders(ctx context.Context) (map[string]string, error) + } + // TrafficController is getting called in the interceptor chain with API invocation parameters. + // Result is either nil if API call is allowed or an error, in which case request would be interrupted and + // the error will be propagated back through the interceptor chain. + TrafficController interface { + CheckCallAllowed(ctx context.Context, method string, req, reply interface{}) error + } + + // ConnectionOptions is provided by SDK consumers to control optional connection params. + ConnectionOptions struct { + // TLS configures connection level security credentials. + TLS *tls.Config + + // Authority specifies the value to be used as the :authority pseudo-header. + // This value only used when TLS is nil. + Authority string + + // Disable keep alive ping from client to the server. + DisableKeepAliveCheck bool + + // After a duration of this time if the client doesn't see any activity it + // pings the server to see if the transport is still alive. + // If set below 10s, a minimum value of 10s will be used instead. + // default: 30s + KeepAliveTime time.Duration + + // After having pinged for keepalive check, the client waits for a duration + // of Timeout and if no activity is seen even after that the connection is + // closed. + // default: 15s + KeepAliveTimeout time.Duration + + // GetSystemInfoTimeout is the timeout for the RPC made by the + // client to fetch server capabilities. + GetSystemInfoTimeout time.Duration + + // if true, when there are no active RPCs, Time and Timeout will be ignored and no + // keepalive pings will be sent. + // If false, client sends keepalive pings even with no active RPCs + // default: false + DisableKeepAlivePermitWithoutStream bool + + // MaxPayloadSize is a number of bytes that gRPC would allow to travel to and from server. Defaults to 128 MB. + MaxPayloadSize int + + // Advanced dial options for gRPC connections. These are applied after the internal default dial options are + // applied. Therefore any dial options here may override internal ones. Dial options WithBlock, WithTimeout, + // WithReturnConnectionError, and FailOnNonTempDialError are ignored since [grpc.NewClient] is used. + // + // For gRPC interceptors, internal interceptors such as error handling, metrics, and retrying are done via + // grpc.WithChainUnaryInterceptor. Therefore to add inner interceptors that are wrapped by those, a + // grpc.WithChainUnaryInterceptor can be added as an option here. To add a single outer interceptor, a + // grpc.WithUnaryInterceptor option can be added since grpc.WithUnaryInterceptor is prepended to chains set with + // grpc.WithChainUnaryInterceptor. + DialOptions []grpc.DialOption + + // Hidden for use by client overloads. + disableEagerConnection bool + + // Internal atomic that, when true, will not retry internal errors like + // other gRPC errors. If not present during service client creation, it will + // be created as false. This is set to true when server capabilities are + // fetched. + excludeInternalFromRetry *atomic.Bool + } + + // StartWorkflowOptions configuration parameters for starting a workflow execution. + // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is + // subjected to change in the future. + StartWorkflowOptions struct { + // ID - The business identifier of the workflow execution. + // Optional: defaulted to a uuid. + ID string + + // TaskQueue - The workflow tasks of the workflow are scheduled on the queue with this name. + // This is also the name of the activity task queue on which activities are scheduled. + // The workflow author can choose to override this using activity options. + // Mandatory: No default. + TaskQueue string + + // WorkflowExecutionTimeout - The timeout for duration of workflow execution. + // It includes retries and continue as new. Use WorkflowRunTimeout to limit execution time + // of a single workflow run. + // The resolution is seconds. + // Optional: defaulted to unlimited. + WorkflowExecutionTimeout time.Duration + + // WorkflowRunTimeout - The timeout for duration of a single workflow run. + // The resolution is seconds. + // Optional: defaulted to WorkflowExecutionTimeout. + WorkflowRunTimeout time.Duration + + // WorkflowTaskTimeout - The timeout for processing workflow task from the time the worker + // pulled this task. If a workflow task is lost, it is retried after this timeout. + // The resolution is seconds. + // Optional: defaulted to 10 secs. + WorkflowTaskTimeout time.Duration + + // WorkflowIDReusePolicy - Specifies server behavior if a *completed* workflow with the same id exists. + // This can be useful for dedupe logic if set to RejectDuplicate + // Optional: defaulted to AllowDuplicate. + WorkflowIDReusePolicy enumspb.WorkflowIdReusePolicy + + // WorkflowIDConflictPolicy - Specifies server behavior if a *running* workflow with the same id exists. + // This cannot be set if WorkflowIDReusePolicy is set to TerminateIfRunning. + // Optional: defaulted to Fail. + WorkflowIDConflictPolicy enumspb.WorkflowIdConflictPolicy + + // WithStartOperation - Operation to execute with Workflow Start. + // For example, see NewUpdateWithStartWorkflowOperation to perform Update-with-Start. Note that if the workflow is + // already running and WorkflowIDConflictPolicy is set to UseExisting, the start is skipped and only the + // operation is executed. If instead the policy is set to Fail (the default), nothing is executed and + // an error will be returned (i.e. the option WorkflowExecutionErrorWhenAlreadyStarted is ignored). + // This option will be ignored when used with Client.SignalWithStartWorkflow. + // + // Optional: defaults to nil. + // + // NOTE: Experimental + WithStartOperation WithStartWorkflowOperation + + // When WorkflowExecutionErrorWhenAlreadyStarted is true, Client.ExecuteWorkflow will return an error if the + // workflow id has already been used and WorkflowIDReusePolicy or WorkflowIDConflictPolicy would + // disallow a re-run. If it is set to false, rather than erroring a WorkflowRun instance representing + // the current or last run will be returned. However, when WithStartOperation is set, this field is ignored and + // the WorkflowIDConflictPolicy UseExisting must be used instead to prevent erroring. + // + // Optional: defaults to false + WorkflowExecutionErrorWhenAlreadyStarted bool + + // RetryPolicy - Optional retry policy for workflow. If a retry policy is specified, in case of workflow failure + // server will start new workflow execution if needed based on the retry policy. + RetryPolicy *RetryPolicy + + // CronSchedule - Optional cron schedule for workflow. If a cron schedule is specified, the workflow will run + // as a cron based on the schedule. The scheduling will be based on UTC time. Schedule for next run only happen + // after the current run is completed/failed/timeout. If a RetryPolicy is also supplied, and the workflow failed + // or timeout, the workflow will be retried based on the retry policy. While the workflow is retrying, it won't + // schedule its next run. If next schedule is due while workflow is running (or retrying), then it will skip that + // schedule. Cron workflow will not stop until it is terminated or canceled (by returning temporal.CanceledError). + // The cron spec is as following: + // ┌───────────── minute (0 - 59) + // │ ┌───────────── hour (0 - 23) + // │ │ ┌───────────── day of the month (1 - 31) + // │ │ │ ┌───────────── month (1 - 12) + // │ │ │ │ ┌───────────── day of the week (0 - 6) (Sunday to Saturday) + // │ │ │ │ │ + // │ │ │ │ │ + // * * * * * + CronSchedule string + + // Memo - Optional non-indexed info that will be shown in list workflow. + Memo map[string]interface{} + + // SearchAttributes - Optional indexed info that can be used in query of List/Scan/Count workflow APIs. The key and value type must be registered on Temporal server side. + // Use GetSearchAttributes API to get valid key and corresponding value type. + // For supported operations on different server versions see [Visibility]. + // + // Deprecated: use TypedSearchAttributes instead. + // + // [Visibility]: https://docs.temporal.io/visibility + SearchAttributes map[string]interface{} + + // TypedSearchAttributes - Specifies Search Attributes that will be attached to the Workflow. Search Attributes are + // additional indexed information attributed to workflow and used for search and visibility. The search attributes + // can be used in query of List/Scan/Count workflow APIs. The key and its value type must be registered on Temporal + // server side. For supported operations on different server versions see [Visibility]. + // + // Optional: default to none. + // + // [Visibility]: https://docs.temporal.io/visibility + TypedSearchAttributes SearchAttributes + + // EnableEagerStart - request eager execution for this workflow, if a local worker is available. + // + // WARNING: Eager start does not respect worker versioning. An eagerly started workflow may run on + // any available local worker even if that worker is not in the default build ID set. + // + // NOTE: Experimental + EnableEagerStart bool + + // StartDelay - Time to wait before dispatching the first workflow task. + // A signal from signal with start will not trigger a workflow task. + // Cannot be set the same time as a CronSchedule. + StartDelay time.Duration + + // StaticSummary - Single-line fixed summary for this workflow execution that will appear in UI/CLI. This can be + // in single-line Temporal markdown format. + // + // Optional: defaults to none/empty. + // + // NOTE: Experimental + StaticSummary string + + // Details - General fixed details for this workflow execution that will appear in UI/CLI. This can be in + // Temporal markdown format and can span multiple lines. This is a fixed value on the workflow that cannot be + // updated. For details that can be updated, use SetCurrentDetails within the workflow. + // + // Optional: defaults to none/empty. + // + // NOTE: Experimental + StaticDetails string + + // request ID. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. + requestID string + // workflow completion callback. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. + callbacks []*commonpb.Callback + // links. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. + links []*commonpb.Link + } + + // WithStartWorkflowOperation is a type of operation that can be executed as part of a workflow start. + WithStartWorkflowOperation interface { + isWithStartWorkflowOperation() + } + + // UpdateWithStartWorkflowOperation is used to perform Update-with-Start. + // See NewUpdateWithStartWorkflowOperation for details. + UpdateWithStartWorkflowOperation struct { + input *ClientUpdateWorkflowInput + // flag to ensure the operation is only executed once + executed atomic.Bool + // channel to indicate that handle or err is available + doneCh chan struct{} + // handle and err cannot be accessed before doneCh is closed + handle WorkflowUpdateHandle + err error + } + + // RetryPolicy defines the retry policy. + // Note that the history of activity with retry policy will be different: the started event will be written down into + // history only when the activity completes or "finally" timeouts/fails. And the started event only records the last + // started time. Because of that, to check an activity has started or not, you cannot rely on history events. Instead, + // you can use CLI to describe the workflow to see the status of the activity: + // tctl --ns wf desc -w + RetryPolicy struct { + // Backoff interval for the first retry. If BackoffCoefficient is 1.0 then it is used for all retries. + // If not set or set to 0, a default interval of 1s will be used. + InitialInterval time.Duration + + // Coefficient used to calculate the next retry backoff interval. + // The next retry interval is previous interval multiplied by this coefficient. + // Must be 1 or larger. Default is 2.0. + BackoffCoefficient float64 + + // Maximum backoff interval between retries. Exponential backoff leads to interval increase. + // This value is the cap of the interval. Default is 100x of initial interval. + MaximumInterval time.Duration + + // Maximum number of attempts. When exceeded the retries stop even if not expired yet. + // If not set or set to 0, it means unlimited, and rely on activity ScheduleToCloseTimeout to stop. + MaximumAttempts int32 + + // Non-Retriable errors. This is optional. Temporal server will stop retry if error type matches this list. + // Note: + // - cancellation is not a failure, so it won't be retried, + // - only StartToClose or Heartbeat timeouts are retryable. + NonRetryableErrorTypes []string + } + + // NamespaceClient is the client for managing operations on the namespace. + // CLI, tools, ... can use this layer to manager operations on namespace. + NamespaceClient interface { + // Register a namespace with temporal server + // The errors it can throw: + // - NamespaceAlreadyExistsError + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + Register(ctx context.Context, request *workflowservice.RegisterNamespaceRequest) error + + // Describe a namespace. The namespace has 3 part of information + // NamespaceInfo - Which has Name, Status, Description, Owner Email + // NamespaceConfiguration - Configuration like Workflow Execution Retention Period In Days, Whether to emit metrics. + // ReplicationConfiguration - replication config like clusters and active cluster name + // The errors it can throw: + // - serviceerror.NamespaceNotFound + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + Describe(ctx context.Context, name string) (*workflowservice.DescribeNamespaceResponse, error) + + // Update a namespace. + // The errors it can throw: + // - serviceerror.NamespaceNotFound + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + Update(ctx context.Context, request *workflowservice.UpdateNamespaceRequest) error + + // Close client and clean up underlying resources. + Close() + } +) + +// Credentials are optional credentials that can be specified in ClientOptions. +type Credentials interface { + applyToOptions(*ConnectionOptions) error + // Can return nil to have no interceptor + gRPCInterceptor() grpc.UnaryClientInterceptor +} + +// DialClient creates a client and attempts to connect to the server. +func DialClient(ctx context.Context, options ClientOptions) (Client, error) { + options.ConnectionOptions.disableEagerConnection = false + return NewClient(ctx, options) +} + +// NewLazyClient creates a client and does not attempt to connect to the server. +func NewLazyClient(options ClientOptions) (Client, error) { + options.ConnectionOptions.disableEagerConnection = true + return NewClient(context.Background(), options) +} + +// NewClient creates an instance of a workflow client +// +// Deprecated: Use DialClient or NewLazyClient instead. +func NewClient(ctx context.Context, options ClientOptions) (Client, error) { + return newClient(ctx, options, nil) +} + +// NewClientFromExisting creates a new client using the same connection as the +// existing client. +func NewClientFromExisting(ctx context.Context, existingClient Client, options ClientOptions) (Client, error) { + existing, _ := existingClient.(*WorkflowClient) + if existing == nil { + return nil, fmt.Errorf("existing client must have been created directly from a client package call") + } + return newClient(ctx, options, existing) +} + +func newClient(ctx context.Context, options ClientOptions, existing *WorkflowClient) (Client, error) { + if options.Namespace == "" { + options.Namespace = DefaultNamespace + } + + // Initialize root tags + if options.MetricsHandler == nil { + options.MetricsHandler = metrics.NopHandler + } + options.MetricsHandler = options.MetricsHandler.WithTags(metrics.RootTags(options.Namespace)) + + if options.HostPort == "" { + options.HostPort = LocalHostPort + } + + if options.Logger == nil { + options.Logger = ilog.NewDefaultLogger() + options.Logger.Info("No logger configured for temporal client. Created default one.") + } + + if options.Credentials != nil { + if err := options.Credentials.applyToOptions(&options.ConnectionOptions); err != nil { + return nil, err + } + } + + // Dial or use existing connection + var connection *grpc.ClientConn + var err error + if existing == nil { + options.ConnectionOptions.excludeInternalFromRetry = &atomic.Bool{} + connection, err = dial(newDialParameters(&options, options.ConnectionOptions.excludeInternalFromRetry)) + if err != nil { + return nil, err + } + } else { + connection = existing.conn + } + + client := NewServiceClient(workflowservice.NewWorkflowServiceClient(connection), connection, options) + + // If using existing connection, always load its capabilities and use them for + // the new connection. Otherwise, only load server capabilities eagerly if not + // disabled. + if existing != nil { + if client.capabilities, err = existing.loadCapabilities(ctx, options.ConnectionOptions.GetSystemInfoTimeout); err != nil { + return nil, err + } + client.unclosedClients = existing.unclosedClients + } else { + if !options.ConnectionOptions.disableEagerConnection { + if _, err := client.loadCapabilities(ctx, options.ConnectionOptions.GetSystemInfoTimeout); err != nil { + client.Close() + return nil, err + } + } + var unclosedClients int32 + client.unclosedClients = &unclosedClients + } + atomic.AddInt32(client.unclosedClients, 1) + + return client, nil +} + +func newDialParameters(options *ClientOptions, excludeInternalFromRetry *atomic.Bool) dialParameters { + return dialParameters{ + UserConnectionOptions: options.ConnectionOptions, + HostPort: options.HostPort, + RequiredInterceptors: requiredInterceptors(options, excludeInternalFromRetry), + DefaultServiceConfig: defaultServiceConfig, + } +} + +// NewServiceClient creates workflow client from workflowservice.WorkflowServiceClient. Must be used internally in unit tests only. +func NewServiceClient(workflowServiceClient workflowservice.WorkflowServiceClient, conn *grpc.ClientConn, options ClientOptions) *WorkflowClient { + // Namespace can be empty in unit tests. + if options.Namespace == "" { + options.Namespace = DefaultNamespace + } + + if options.Identity == "" { + options.Identity = getWorkerIdentity("") + } + + if options.DataConverter == nil { + options.DataConverter = converter.GetDefaultDataConverter() + } + + if options.FailureConverter == nil { + options.FailureConverter = GetDefaultFailureConverter() + } + + if options.MetricsHandler == nil { + options.MetricsHandler = metrics.NopHandler + } + + if options.ConnectionOptions.excludeInternalFromRetry == nil { + options.ConnectionOptions.excludeInternalFromRetry = &atomic.Bool{} + } + + // Collect set of applicable worker interceptors + var workerInterceptors []WorkerInterceptor + for _, interceptor := range options.Interceptors { + if workerInterceptor, _ := interceptor.(WorkerInterceptor); workerInterceptor != nil { + workerInterceptors = append(workerInterceptors, workerInterceptor) + } + } + + client := &WorkflowClient{ + workflowService: workflowServiceClient, + conn: conn, + namespace: options.Namespace, + registry: newRegistry(), + metricsHandler: options.MetricsHandler, + logger: options.Logger, + identity: options.Identity, + dataConverter: options.DataConverter, + failureConverter: options.FailureConverter, + contextPropagators: options.ContextPropagators, + workerInterceptors: workerInterceptors, + excludeInternalFromRetry: options.ConnectionOptions.excludeInternalFromRetry, + eagerDispatcher: &eagerWorkflowDispatcher{ + workersByTaskQueue: make(map[string][]eagerWorker), + }, + } + + // Create outbound interceptor by wrapping backwards through chain + client.interceptor = &workflowClientInterceptor{client: client} + for i := len(options.Interceptors) - 1; i >= 0; i-- { + client.interceptor = options.Interceptors[i].InterceptClient(client.interceptor) + } + + return client +} + +// DialCloudOperationsClient creates a cloud client to perform cloud-management +// operations. +func DialCloudOperationsClient(ctx context.Context, options CloudOperationsClientOptions) (CloudOperationsClient, error) { + // Set defaults + if options.MetricsHandler == nil { + options.MetricsHandler = metrics.NopHandler + } + if options.Logger == nil { + options.Logger = ilog.NewDefaultLogger() + } + if options.HostPort == "" { + options.HostPort = "saas-api.tmprl.cloud:443" + } + if options.Version != "" { + options.ConnectionOptions.DialOptions = append( + options.ConnectionOptions.DialOptions, + grpc.WithChainUnaryInterceptor(func( + ctx context.Context, method string, req, reply any, + cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption, + ) error { + ctx = metadata.AppendToOutgoingContext(ctx, "temporal-cloud-api-version", options.Version) + return invoker(ctx, method, req, reply, cc, opts...) + }), + ) + } + if options.Credentials != nil { + if err := options.Credentials.applyToOptions(&options.ConnectionOptions); err != nil { + return nil, err + } + } + if options.ConnectionOptions.TLS == nil && !options.DisableTLS { + options.ConnectionOptions.TLS = &tls.Config{} + } + // Exclude internal from retry by default + options.ConnectionOptions.excludeInternalFromRetry = &atomic.Bool{} + options.ConnectionOptions.excludeInternalFromRetry.Store(true) + // TODO(cretz): Pass through context on dial + conn, err := dial(newDialParameters(&ClientOptions{ + HostPort: options.HostPort, + ConnectionOptions: options.ConnectionOptions, + MetricsHandler: options.MetricsHandler, + Credentials: options.Credentials, + }, options.ConnectionOptions.excludeInternalFromRetry)) + if err != nil { + return nil, err + } + return &cloudOperationsClient{ + conn: conn, + logger: options.Logger, + cloudServiceClient: cloudservice.NewCloudServiceClient(conn), + }, nil +} + +// NewUpdateWithStartWorkflowOperation returns an UpdateWithStartWorkflowOperation that can be used to perform Update-with-Start. +func NewUpdateWithStartWorkflowOperation(options UpdateWorkflowOptions) *UpdateWithStartWorkflowOperation { + res := &UpdateWithStartWorkflowOperation{doneCh: make(chan struct{})} + + input, err := createUpdateWorkflowInput(options) + if err != nil { + res.set(nil, err) + } else if options.RunID != "" { + res.set(nil, errors.New("RunID cannot be set because the workflow might not be running")) + } + if options.FirstExecutionRunID != "" { + res.set(nil, errors.New("FirstExecutionRunID cannot be set because the workflow might not be running")) + } else { + res.input = input + } + + return res +} + +// Get blocks until a server response has been received; or the context deadline is exceeded. +func (op *UpdateWithStartWorkflowOperation) Get(ctx context.Context) (WorkflowUpdateHandle, error) { + select { + case <-op.doneCh: + return op.handle, op.err + case <-ctx.Done(): + return nil, ctx.Err() + } +} + +func (op *UpdateWithStartWorkflowOperation) markExecuted() error { + if op.executed.Swap(true) { + return fmt.Errorf("was already executed") + } + return nil +} + +func (op *UpdateWithStartWorkflowOperation) set(handle WorkflowUpdateHandle, err error) { + op.handle = handle + op.err = err + close(op.doneCh) +} + +func (op *UpdateWithStartWorkflowOperation) isWithStartWorkflowOperation() {} + +// NewNamespaceClient creates an instance of a namespace client, to manager lifecycle of namespaces. +func NewNamespaceClient(options ClientOptions) (NamespaceClient, error) { + // Initialize root tags + if options.MetricsHandler == nil { + options.MetricsHandler = metrics.NopHandler + } + options.MetricsHandler = options.MetricsHandler.WithTags(metrics.RootTags(metrics.NoneTagValue)) + + if options.HostPort == "" { + options.HostPort = LocalHostPort + } + + connection, err := dial(newDialParameters(&options, nil)) + if err != nil { + return nil, err + } + + return newNamespaceServiceClient(workflowservice.NewWorkflowServiceClient(connection), connection, options), nil +} + +func newNamespaceServiceClient(workflowServiceClient workflowservice.WorkflowServiceClient, clientConn *grpc.ClientConn, options ClientOptions) NamespaceClient { + if options.Identity == "" { + options.Identity = getWorkerIdentity("") + } + + return &namespaceClient{ + workflowService: workflowServiceClient, + connectionCloser: clientConn, + metricsHandler: options.MetricsHandler, + logger: options.Logger, + identity: options.Identity, + } +} + +// NewValue creates a new converter.EncodedValue which can be used to decode binary data returned by Temporal. For example: +// User had Activity.RecordHeartbeat(ctx, "my-heartbeat") and then got response from calling Client.DescribeWorkflowExecution. +// The response contains binary field PendingActivityInfo.HeartbeatDetails, +// which can be decoded by using: +// +// var result string // This need to be same type as the one passed to RecordHeartbeat +// NewValue(data).Get(&result) +func NewValue(data *commonpb.Payloads) converter.EncodedValue { + return newEncodedValue(data, nil) +} + +// NewValues creates a new converter.EncodedValues which can be used to decode binary data returned by Temporal. For example: +// User had Activity.RecordHeartbeat(ctx, "my-heartbeat", 123) and then got response from calling Client.DescribeWorkflowExecution. +// The response contains binary field PendingActivityInfo.HeartbeatDetails, +// which can be decoded by using: +// +// var result1 string +// var result2 int // These need to be same type as those arguments passed to RecordHeartbeat +// NewValues(data).Get(&result1, &result2) +func NewValues(data *commonpb.Payloads) converter.EncodedValues { + return newEncodedValues(data, nil) +} + +type apiKeyCredentials func(context.Context) (string, error) + +func NewAPIKeyStaticCredentials(apiKey string) Credentials { + return NewAPIKeyDynamicCredentials(func(ctx context.Context) (string, error) { return apiKey, nil }) +} + +func NewAPIKeyDynamicCredentials(apiKeyCallback func(context.Context) (string, error)) Credentials { + return apiKeyCredentials(apiKeyCallback) +} + +func (apiKeyCredentials) applyToOptions(*ConnectionOptions) error { return nil } + +func (a apiKeyCredentials) gRPCInterceptor() grpc.UnaryClientInterceptor { return a.gRPCIntercept } + +func (a apiKeyCredentials) gRPCIntercept( + ctx context.Context, + method string, + req any, + reply any, + cc *grpc.ClientConn, + invoker grpc.UnaryInvoker, + opts ...grpc.CallOption, +) error { + if apiKey, err := a(ctx); err != nil { + return err + } else if apiKey != "" { + // Only add API key if it doesn't already exist + if md, _ := metadata.FromOutgoingContext(ctx); len(md.Get("authorization")) == 0 { + ctx = metadata.AppendToOutgoingContext(ctx, "authorization", "Bearer "+apiKey) + } + } + return invoker(ctx, method, req, reply, cc, opts...) +} + +type mTLSCredentials tls.Certificate + +func NewMTLSCredentials(certificate tls.Certificate) Credentials { return mTLSCredentials(certificate) } + +func (m mTLSCredentials) applyToOptions(opts *ConnectionOptions) error { + if opts.TLS == nil { + opts.TLS = &tls.Config{} + } else if len(opts.TLS.Certificates) != 0 { + return fmt.Errorf("cannot apply mTLS credentials, certificates already exist on TLS options") + } + opts.TLS.Certificates = append(opts.TLS.Certificates, tls.Certificate(m)) + return nil +} + +func (mTLSCredentials) gRPCInterceptor() grpc.UnaryClientInterceptor { return nil } + +// WorkflowUpdateServiceTimeoutOrCanceledError is an error that occurs when an update call times out or is cancelled. +// +// Note, this is not related to any general concept of timing out or cancelling a running update, this is only related to the client call itself. +type WorkflowUpdateServiceTimeoutOrCanceledError struct { + cause error +} + +// NewWorkflowUpdateServiceTimeoutOrCanceledError creates a new WorkflowUpdateServiceTimeoutOrCanceledError. +func NewWorkflowUpdateServiceTimeoutOrCanceledError(err error) *WorkflowUpdateServiceTimeoutOrCanceledError { + return &WorkflowUpdateServiceTimeoutOrCanceledError{ + cause: err, + } +} + +func (e *WorkflowUpdateServiceTimeoutOrCanceledError) Error() string { + return fmt.Sprintf("Timeout or cancellation waiting for update: %v", e.cause) +} + +func (e *WorkflowUpdateServiceTimeoutOrCanceledError) Unwrap() error { return e.cause } + +// SetRequestIDOnStartWorkflowOptions is an internal only method for setting a requestID on StartWorkflowOptions. +// RequestID is purposefully not exposed to users for the time being. +func SetRequestIDOnStartWorkflowOptions(opts *StartWorkflowOptions, requestID string) { + opts.requestID = requestID +} + +// SetCallbacksOnStartWorkflowOptions is an internal only method for setting callbacks on StartWorkflowOptions. +// Callbacks are purposefully not exposed to users for the time being. +func SetCallbacksOnStartWorkflowOptions(opts *StartWorkflowOptions, callbacks []*commonpb.Callback) { + opts.callbacks = callbacks +} + +// SetLinksOnStartWorkflowOptions is an internal only method for setting links on StartWorkflowOptions. +// Links are purposefully not exposed to users for the time being. +func SetLinksOnStartWorkflowOptions(opts *StartWorkflowOptions, links []*commonpb.Link) { + opts.links = links +} diff --git a/vendor/go.temporal.io/sdk/internal/common/backoff/retry.go b/vendor/go.temporal.io/sdk/internal/common/backoff/retry.go new file mode 100644 index 00000000000..e37c484676e --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/backoff/retry.go @@ -0,0 +1,196 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package backoff + +import ( + "context" + "errors" + "sync" + "time" + + "go.temporal.io/api/serviceerror" +) + +type ( + // Operation to retry + Operation func() error + + // IsRetryable handler can be used to exclude certain errors during retry + IsRetryable func(error) bool + + // ConcurrentRetrier is used for client-side throttling. It determines whether to + // throttle outgoing traffic in case downstream backend server rejects + // requests due to out-of-quota or server busy errors. + ConcurrentRetrier struct { + sync.Mutex + retrier Retrier // Backoff retrier + secondaryRetrier Retrier + failureCount int64 // Number of consecutive failures seen + includeSecondaryRetrier bool + } +) + +// Throttle Sleep if there were failures since the last success call. The +// provided done channel provides a way to exit early. +func (c *ConcurrentRetrier) Throttle(doneCh <-chan struct{}) { + c.throttleInternal(doneCh) +} + +// GetElapsedTime gets the amount of time since that last ConcurrentRetrier.Succeeded call +func (c *ConcurrentRetrier) GetElapsedTime() time.Duration { + c.Lock() + defer c.Unlock() + return c.retrier.GetElapsedTime() +} + +func (c *ConcurrentRetrier) throttleInternal(doneCh <-chan struct{}) time.Duration { + next := done + + // Check if we have failure count. + c.Lock() + if c.failureCount > 0 { + next = c.retrier.NextBackOff() + // If secondary is included, use the greatest of the two (which also means + // if one is "done", which is -1, the one that's not done is chosen) + if c.includeSecondaryRetrier { + c.includeSecondaryRetrier = false + if c.secondaryRetrier != nil { + if secNext := c.secondaryRetrier.NextBackOff(); secNext > next { + next = secNext + } + } + } + } + c.Unlock() + + if next != done { + select { + case <-doneCh: + case <-time.After(next): + } + } + + return next +} + +// Succeeded marks client request succeeded. +func (c *ConcurrentRetrier) Succeeded() { + defer c.Unlock() + c.Lock() + c.failureCount = 0 + c.includeSecondaryRetrier = false + c.retrier.Reset() + if c.secondaryRetrier != nil { + c.secondaryRetrier.Reset() + } +} + +// Failed marks client request failed because backend is busy. If +// includeSecondaryRetryPolicy is true, see SetSecondaryRetryPolicy for effects. +func (c *ConcurrentRetrier) Failed(includeSecondaryRetryPolicy bool) { + defer c.Unlock() + c.Lock() + c.failureCount++ + c.includeSecondaryRetrier = includeSecondaryRetryPolicy +} + +// SetSecondaryRetryPolicy sets a secondary retry policy that, if Failed is +// called with true, will trigger the secondary retry policy in addition to the +// primary and will use the result of the secondary if longer than the primary. +func (c *ConcurrentRetrier) SetSecondaryRetryPolicy(retryPolicy RetryPolicy) { + c.Lock() + defer c.Unlock() + if retryPolicy == nil { + c.secondaryRetrier = nil + } else { + c.secondaryRetrier = NewRetrier(retryPolicy, SystemClock) + } +} + +// NewConcurrentRetrier returns an instance of concurrent backoff retrier. +func NewConcurrentRetrier(retryPolicy RetryPolicy) *ConcurrentRetrier { + retrier := NewRetrier(retryPolicy, SystemClock) + return &ConcurrentRetrier{retrier: retrier} +} + +// Retry function can be used to wrap any call with retry logic using the passed in policy +func Retry(ctx context.Context, operation Operation, policy RetryPolicy, isRetryable IsRetryable) error { + var lastErr error + var next time.Duration + + r := NewRetrier(policy, SystemClock) + for { + opErr := operation() + if opErr == nil { + // operation completed successfully. No need to retry. + return nil + } + + // Usually, after number of retry attempts, last attempt fails with DeadlineExceeded error. + // It is not informative and actual error reason is in the error occurred on previous attempt. + // Therefore, update lastErr only if it is not set (first attempt) or opErr is not a DeadlineExceeded error. + // This lastErr is returned if retry attempts are exhausted. + var errDeadlineExceeded *serviceerror.DeadlineExceeded + if lastErr == nil || !(errors.Is(opErr, context.DeadlineExceeded) || errors.As(opErr, &errDeadlineExceeded)) { + lastErr = opErr + } + + if next = r.NextBackOff(); next == done { + return lastErr + } + + // Check if the error is retryable + if isRetryable != nil && !isRetryable(opErr) { + return lastErr + } + + // check if ctx is done + if ctxDone := ctx.Done(); ctxDone != nil { + timer := time.NewTimer(next) + select { + case <-ctxDone: + return lastErr + case <-timer.C: + continue + } + } + + // ctx is not cancellable + time.Sleep(next) + } +} + +// IgnoreErrors can be used as IsRetryable handler for Retry function to exclude certain errors from the retry list +func IgnoreErrors(errorsToExclude []error) func(error) bool { + return func(err error) bool { + for _, errorToExclude := range errorsToExclude { + if errors.Is(err, errorToExclude) { + return false + } + } + + return true + } +} diff --git a/vendor/go.temporal.io/sdk/internal/common/backoff/retrypolicy.go b/vendor/go.temporal.io/sdk/internal/common/backoff/retrypolicy.go new file mode 100644 index 00000000000..dd4e7791c7b --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/backoff/retrypolicy.go @@ -0,0 +1,210 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package backoff + +import ( + "math" + "math/rand" + "time" + + "go.temporal.io/sdk/internal/common/retry" +) + +const ( + done time.Duration = -1 +) + +type ( + // RetryPolicy is the API which needs to be implemented by various retry policy implementations + RetryPolicy interface { + ComputeNextDelay(elapsedTime time.Duration, attempt int) time.Duration + GrpcRetryConfig() *retry.GrpcRetryConfig + } + + // Retrier manages the state of retry operation + Retrier interface { + GetElapsedTime() time.Duration + NextBackOff() time.Duration + Reset() + } + + // Clock used by ExponentialRetryPolicy implementation to get the current time. Mainly used for unit testing + Clock interface { + Now() time.Time + } + + // ExponentialRetryPolicy provides the implementation for retry policy using a coefficient to compute the next delay. + // Formula used to compute the next delay is: initialInterval * math.Pow(backoffCoefficient, currentAttempt) + ExponentialRetryPolicy struct { + initialInterval time.Duration + backoffCoefficient float64 + maximumInterval time.Duration + expirationInterval time.Duration + maximumAttempts int + } + + systemClock struct{} + + retrierImpl struct { + policy RetryPolicy + clock Clock + currentAttempt int + startTime time.Time + } +) + +// SystemClock implements Clock interface that uses time.Now(). +var SystemClock = systemClock{} + +// NewExponentialRetryPolicy returns an instance of ExponentialRetryPolicy using the provided initialInterval +func NewExponentialRetryPolicy(initialInterval time.Duration) *ExponentialRetryPolicy { + p := &ExponentialRetryPolicy{ + initialInterval: initialInterval, + backoffCoefficient: retry.DefaultBackoffCoefficient, + maximumInterval: retry.DefaultMaximumInterval, + expirationInterval: retry.DefaultExpirationInterval, + maximumAttempts: retry.DefaultMaximumAttempts, + } + + return p +} + +// NewRetrier is used for creating a new instance of Retrier +func NewRetrier(policy RetryPolicy, clock Clock) Retrier { + return &retrierImpl{ + policy: policy, + clock: clock, + startTime: clock.Now(), + currentAttempt: 1, + } +} + +// SetInitialInterval sets the initial interval used by ExponentialRetryPolicy for the very first retry +// All later retries are computed using the following formula: +// initialInterval * math.Pow(backoffCoefficient, currentAttempt) +func (p *ExponentialRetryPolicy) SetInitialInterval(initialInterval time.Duration) { + p.initialInterval = initialInterval +} + +// SetBackoffCoefficient sets the coefficient used by ExponentialRetryPolicy to compute next delay for each retry +// All retries are computed using the following formula: +// initialInterval * math.Pow(backoffCoefficient, currentAttempt) +func (p *ExponentialRetryPolicy) SetBackoffCoefficient(backoffCoefficient float64) { + p.backoffCoefficient = backoffCoefficient +} + +// SetMaximumInterval sets the maximum interval for each retry +func (p *ExponentialRetryPolicy) SetMaximumInterval(maximumInterval time.Duration) { + p.maximumInterval = maximumInterval +} + +// SetExpirationInterval sets the absolute expiration interval for all retries +func (p *ExponentialRetryPolicy) SetExpirationInterval(expirationInterval time.Duration) { + p.expirationInterval = expirationInterval +} + +// SetMaximumAttempts sets the maximum number of retry attempts +func (p *ExponentialRetryPolicy) SetMaximumAttempts(maximumAttempts int) { + p.maximumAttempts = maximumAttempts +} + +// ComputeNextDelay returns the next delay interval. This is used by Retrier to delay calling the operation again +func (p *ExponentialRetryPolicy) ComputeNextDelay(elapsedTime time.Duration, attempt int) time.Duration { + // Check to see if we ran out of maximum number of attempts + if p.maximumAttempts != retry.UnlimitedMaximumAttempts && attempt >= p.maximumAttempts { + return done + } + + // Stop retrying after expiration interval is elapsed + if p.expirationInterval != retry.UnlimitedInterval && elapsedTime > p.expirationInterval { + return done + } + + nextInterval := float64(p.initialInterval) * math.Pow(p.backoffCoefficient, float64(attempt-1)) + // Disallow retries if initialInterval is negative or nextInterval overflows + if nextInterval <= 0 { + return done + } + if p.maximumInterval != retry.UnlimitedInterval { + nextInterval = math.Min(nextInterval, float64(p.maximumInterval)) + } + + if p.expirationInterval != retry.UnlimitedInterval { + remainingTime := math.Max(0, float64(p.expirationInterval-elapsedTime)) + nextInterval = math.Min(remainingTime, nextInterval) + } + + // Bail out if the next interval is smaller than initial retry interval + nextDuration := time.Duration(nextInterval) + if nextDuration < p.initialInterval { + return done + } + + // add jitter to avoid global synchronization + jitterPortion := int(retry.DefaultJitter * nextInterval) + // Prevent overflow + if jitterPortion < 1 { + jitterPortion = 1 + } + nextInterval = nextInterval*(1-retry.DefaultJitter) + float64(rand.Intn(jitterPortion)) + + return time.Duration(nextInterval) +} + +// GrpcRetryConfig converts retry policy into retry config. +func (p *ExponentialRetryPolicy) GrpcRetryConfig() *retry.GrpcRetryConfig { + retryConfig := retry.NewGrpcRetryConfig(p.initialInterval) + retryConfig.SetBackoffCoefficient(p.backoffCoefficient) + retryConfig.SetExpirationInterval(p.expirationInterval) + retryConfig.SetMaximumAttempts(p.maximumAttempts) + retryConfig.SetMaximumInterval(p.maximumInterval) + return retryConfig +} + +// Now returns the current time using the system clock +func (t systemClock) Now() time.Time { + return time.Now() +} + +// Reset will set the Retrier into initial state +func (r *retrierImpl) Reset() { + r.startTime = r.clock.Now() + r.currentAttempt = 1 +} + +// NextBackOff returns the next delay interval. This is used by Retry to delay calling the operation again +func (r *retrierImpl) NextBackOff() time.Duration { + nextInterval := r.policy.ComputeNextDelay(r.GetElapsedTime(), r.currentAttempt) + + // Now increment the current attempt + r.currentAttempt++ + return nextInterval +} + +// GetElapsedTime returns the amount of time since the retrier was created or the last reset, +// whatever was sooner. +func (r *retrierImpl) GetElapsedTime() time.Duration { + return r.clock.Now().Sub(r.startTime) +} diff --git a/vendor/go.temporal.io/sdk/internal/common/cache/cache.go b/vendor/go.temporal.io/sdk/internal/common/cache/cache.go new file mode 100644 index 00000000000..b06990910de --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/cache/cache.go @@ -0,0 +1,82 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package cache + +import ( + "time" +) + +// A Cache is a generalized interface to a cache. See cache.LRU for a specific +// implementation (bounded cache with LRU eviction) +type Cache interface { + // Exist checks if a given key exists in the cache + Exist(key string) bool + + // Get retrieves an element based on a key, returning nil if the element + // does not exist + Get(key string) interface{} + + // Put adds an element to the cache, returning the previous element + Put(key string, value interface{}) interface{} + + // PutIfNotExist puts a value associated with a given key if it does not exist + PutIfNotExist(key string, value interface{}) (interface{}, error) + + // Delete deletes an element in the cache + Delete(key string) + + // Release decrements the ref count of a pinned element. If the ref count + // drops to 0, the element can be evicted from the cache. + Release(key string) + + // Size returns the number of entries currently stored in the Cache + Size() int + + // Clear clears the cache. + Clear() +} + +// Options control the behavior of the cache +type Options struct { + // TTL controls the time-to-live for a given cache entry. Cache entries that + // are older than the TTL will not be returned + TTL time.Duration + + // InitialCapacity controls the initial capacity of the cache + InitialCapacity int + + // Pin prevents in-use objects from getting evicted + Pin bool + + // RemovedFunc is an optional function called when an element + // is scheduled for deletion + RemovedFunc RemovedFunc +} + +// RemovedFunc is a type for notifying applications when an item is +// scheduled for removal from the Cache. If f is a function with the +// appropriate signature and i is the interface{} scheduled for +// deletion, Cache calls go f(i) +type RemovedFunc func(interface{}) diff --git a/vendor/go.temporal.io/sdk/internal/common/cache/lru.go b/vendor/go.temporal.io/sdk/internal/common/cache/lru.go new file mode 100644 index 00000000000..df88ea50e35 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/cache/lru.go @@ -0,0 +1,255 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package cache + +import ( + "container/list" + "errors" + "sync" + "time" +) + +var ( + // ErrCacheFull is returned if Put fails due to cache being filled with pinned elements + ErrCacheFull = errors.New("Cache capacity is fully occupied with pinned elements") +) + +// lru is a concurrent fixed size cache that evicts elements in lru order +type lru struct { + mut sync.Mutex + byAccess *list.List + byKey map[string]*list.Element + maxSize int + ttl time.Duration + pin bool + rmFunc RemovedFunc +} + +// New creates a new cache with the given options +func New(maxSize int, opts *Options) Cache { + if opts == nil { + opts = &Options{} + } + + return &lru{ + byAccess: list.New(), + byKey: make(map[string]*list.Element, opts.InitialCapacity), + ttl: opts.TTL, + maxSize: maxSize, + pin: opts.Pin, + rmFunc: opts.RemovedFunc, + } +} + +// NewLRU creates a new LRU cache of the given size, setting initial capacity +// to the max size +func NewLRU(maxSize int) Cache { + return New(maxSize, nil) +} + +// NewLRUWithInitialCapacity creates a new LRU cache with an initial capacity +// and a max size +func NewLRUWithInitialCapacity(initialCapacity, maxSize int) Cache { + return New(maxSize, &Options{ + InitialCapacity: initialCapacity, + }) +} + +// Exist checks if a given key exists in the cache +func (c *lru) Exist(key string) bool { + c.mut.Lock() + defer c.mut.Unlock() + _, ok := c.byKey[key] + return ok +} + +// Get retrieves the value stored under the given key +func (c *lru) Get(key string) interface{} { + c.mut.Lock() + defer c.mut.Unlock() + + elt := c.byKey[key] + if elt == nil { + return nil + } + + cacheEntry := elt.Value.(*cacheEntry) + + if c.pin { + cacheEntry.refCount++ + } + + if cacheEntry.refCount == 0 && !cacheEntry.expiration.IsZero() && time.Now().After(cacheEntry.expiration) { + // Entry has expired + if c.rmFunc != nil { + go c.rmFunc(cacheEntry.value) + } + c.byAccess.Remove(elt) + delete(c.byKey, cacheEntry.key) + return nil + } + + c.byAccess.MoveToFront(elt) + return cacheEntry.value +} + +// Put puts a new value associated with a given key, returning the existing value (if present) +func (c *lru) Put(key string, value interface{}) interface{} { + if c.pin { + panic("Cannot use Put API in Pin mode. Use Delete and PutIfNotExist if necessary") + } + val, _ := c.putInternal(key, value, true) + return val +} + +// PutIfNotExist puts a value associated with a given key if it does not exist +func (c *lru) PutIfNotExist(key string, value interface{}) (interface{}, error) { + existing, err := c.putInternal(key, value, false) + if err != nil { + return nil, err + } + + if existing == nil { + // This is a new value + return value, err + } + + return existing, err +} + +// Delete deletes a key, value pair associated with a key +func (c *lru) Delete(key string) { + c.mut.Lock() + defer c.mut.Unlock() + + elt := c.byKey[key] + if elt != nil { + entry := c.byAccess.Remove(elt).(*cacheEntry) + if c.rmFunc != nil { + go c.rmFunc(entry.value) + } + delete(c.byKey, key) + } +} + +// Release decrements the ref count of a pinned element. +func (c *lru) Release(key string) { + c.mut.Lock() + defer c.mut.Unlock() + + elt := c.byKey[key] + cacheEntry := elt.Value.(*cacheEntry) + cacheEntry.refCount-- +} + +// Size returns the number of entries currently in the lru, useful if cache is not full +func (c *lru) Size() int { + c.mut.Lock() + defer c.mut.Unlock() + + return len(c.byKey) +} + +// Clear clears the cache. +func (c *lru) Clear() { + c.mut.Lock() + defer c.mut.Unlock() + + for key, elt := range c.byKey { + if elt != nil { + entry := c.byAccess.Remove(elt).(*cacheEntry) + if c.rmFunc != nil { + go c.rmFunc(entry.value) + } + delete(c.byKey, key) + } + } +} + +// Put puts a new value associated with a given key, returning the existing value (if present) +// allowUpdate flag is used to control overwrite behavior if the value exists +func (c *lru) putInternal(key string, value interface{}, allowUpdate bool) (interface{}, error) { + c.mut.Lock() + defer c.mut.Unlock() + + elt := c.byKey[key] + if elt != nil { + entry := elt.Value.(*cacheEntry) + existing := entry.value + if allowUpdate { + entry.value = value + } + if c.ttl != 0 { + entry.expiration = time.Now().Add(c.ttl) + } + c.byAccess.MoveToFront(elt) + if c.pin { + entry.refCount++ + } + return existing, nil + } + + entry := &cacheEntry{ + key: key, + value: value, + } + + if c.pin { + entry.refCount++ + } + + if c.ttl != 0 { + entry.expiration = time.Now().Add(c.ttl) + } + + c.byKey[key] = c.byAccess.PushFront(entry) + // Only trigger eviction when we have exceeded the max + if len(c.byKey) > c.maxSize { + oldest := c.byAccess.Back().Value.(*cacheEntry) + + if oldest.refCount > 0 { + // Cache is full with pinned elements + // revert the insert and return + c.byAccess.Remove(c.byAccess.Front()) + delete(c.byKey, key) + return nil, ErrCacheFull + } + + c.byAccess.Remove(c.byAccess.Back()) + if c.rmFunc != nil { + go c.rmFunc(oldest.value) + } + delete(c.byKey, oldest.key) + } + + return nil, nil +} + +type cacheEntry struct { + key string + expiration time.Time + value interface{} + refCount int +} diff --git a/vendor/go.temporal.io/sdk/internal/common/metrics/capturing_handler.go b/vendor/go.temporal.io/sdk/internal/common/metrics/capturing_handler.go new file mode 100644 index 00000000000..ac976326916 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/metrics/capturing_handler.go @@ -0,0 +1,222 @@ +// The MIT License +// +// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package metrics + +import ( + "sync" + "sync/atomic" + "time" +) + +// This file contains test helpers only. They are not private because they are used by other tests. + +type capturedInfo struct { + sliceLock sync.RWMutex // Only governs slice access, not what's in the slice + counters []*CapturedCounter + gauges []*CapturedGauge + timers []*CapturedTimer +} + +// CapturingHandler is a Handler that retains counted values locally. +type CapturingHandler struct { + *capturedInfo + // Never changed once created + tags map[string]string +} + +var _ Handler = &CapturingHandler{} + +// NewCapturingHandler creates a new CapturingHandler. +func NewCapturingHandler() *CapturingHandler { return &CapturingHandler{capturedInfo: &capturedInfo{}} } + +// Clear removes all known metrics from the root handler. +func (c *CapturingHandler) Clear() { + c.sliceLock.Lock() + defer c.sliceLock.Unlock() + c.counters = nil + c.gauges = nil + c.timers = nil +} + +// WithTags implements Handler.WithTags. +func (c *CapturingHandler) WithTags(tags map[string]string) Handler { + ret := &CapturingHandler{capturedInfo: c.capturedInfo, tags: make(map[string]string)} + for k, v := range c.tags { + ret.tags[k] = v + } + for k, v := range tags { + ret.tags[k] = v + } + return ret +} + +// Counter implements Handler.Counter. +func (c *CapturingHandler) Counter(name string) Counter { + c.sliceLock.Lock() + defer c.sliceLock.Unlock() + // Try to find one or create otherwise + var ret *CapturedCounter + for _, counter := range c.counters { + if counter.Name == name && counter.equalTags(c.tags) { + ret = counter + break + } + } + if ret == nil { + ret = &CapturedCounter{CapturedMetricMeta: CapturedMetricMeta{Name: name, Tags: c.tags}} + c.counters = append(c.counters, ret) + } + return ret +} + +// Counters returns shallow copy of the local counters. New counters will not +// get added here, but the value within the counter may still change. +func (c *CapturingHandler) Counters() []*CapturedCounter { + c.sliceLock.RLock() + defer c.sliceLock.RUnlock() + ret := make([]*CapturedCounter, len(c.counters)) + copy(ret, c.counters) + return ret +} + +// Gauge implements Handler.Gauge. +func (c *CapturingHandler) Gauge(name string) Gauge { + c.sliceLock.Lock() + defer c.sliceLock.Unlock() + // Try to find one or create otherwise + var ret *CapturedGauge + for _, gauge := range c.gauges { + if gauge.Name == name && gauge.equalTags(c.tags) { + ret = gauge + break + } + } + if ret == nil { + ret = &CapturedGauge{CapturedMetricMeta: CapturedMetricMeta{Name: name, Tags: c.tags}} + c.gauges = append(c.gauges, ret) + } + return ret +} + +// Gauges returns shallow copy of the local gauges. New gauges will not get +// added here, but the value within the gauge may still change. +func (c *CapturingHandler) Gauges() []*CapturedGauge { + c.sliceLock.RLock() + defer c.sliceLock.RUnlock() + ret := make([]*CapturedGauge, len(c.gauges)) + copy(ret, c.gauges) + return ret +} + +// Timer implements Handler.Timer. +func (c *CapturingHandler) Timer(name string) Timer { + c.sliceLock.Lock() + defer c.sliceLock.Unlock() + // Try to find one or create otherwise + var ret *CapturedTimer + for _, timer := range c.timers { + if timer.Name == name && timer.equalTags(c.tags) { + ret = timer + break + } + } + if ret == nil { + ret = &CapturedTimer{CapturedMetricMeta: CapturedMetricMeta{Name: name, Tags: c.tags}} + c.timers = append(c.timers, ret) + } + return ret +} + +// Timers returns shallow copy of the local timers. New timers will not get +// added here, but the value within the timer may still change. +func (c *CapturingHandler) Timers() []*CapturedTimer { + c.sliceLock.RLock() + defer c.sliceLock.RUnlock() + ret := make([]*CapturedTimer, len(c.timers)) + copy(ret, c.timers) + return ret +} + +// CapturedMetricMeta is common information for captured metrics. These fields +// should never by mutated. +type CapturedMetricMeta struct { + Name string + Tags map[string]string +} + +func (c *CapturedMetricMeta) equalTags(other map[string]string) bool { + if len(c.Tags) != len(other) { + return false + } + for k, v := range c.Tags { + if otherV, ok := other[k]; !ok || otherV != v { + return false + } + } + return true +} + +// CapturedCounter atomically implements Counter and provides an atomic getter. +type CapturedCounter struct { + CapturedMetricMeta + value int64 +} + +// Inc implements Counter.Inc. +func (c *CapturedCounter) Inc(d int64) { atomic.AddInt64(&c.value, d) } + +// Value atomically returns the current value. +func (c *CapturedCounter) Value() int64 { return atomic.LoadInt64(&c.value) } + +// CapturedGauge atomically implements Gauge and provides an atomic getter. +type CapturedGauge struct { + CapturedMetricMeta + value float64 + valueLock sync.RWMutex +} + +// Update implements Gauge.Update. +func (c *CapturedGauge) Update(d float64) { + c.valueLock.Lock() + defer c.valueLock.Unlock() + c.value = d +} + +// Value atomically returns the current value. +func (c *CapturedGauge) Value() float64 { + c.valueLock.RLock() + defer c.valueLock.RUnlock() + return c.value +} + +// CapturedTimer atomically implements Timer and provides an atomic getter. +type CapturedTimer struct { + CapturedMetricMeta + value int64 +} + +// Record implements Timer.Record. +func (c *CapturedTimer) Record(d time.Duration) { atomic.StoreInt64(&c.value, int64(d)) } + +// Value atomically returns the current value. +func (c *CapturedTimer) Value() time.Duration { return time.Duration(atomic.LoadInt64(&c.value)) } diff --git a/vendor/go.temporal.io/sdk/internal/common/metrics/constants.go b/vendor/go.temporal.io/sdk/internal/common/metrics/constants.go new file mode 100644 index 00000000000..97b93bed376 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/metrics/constants.go @@ -0,0 +1,118 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package metrics + +// Metrics keys +const ( + TemporalMetricsPrefix = "temporal_" + + WorkflowCompletedCounter = TemporalMetricsPrefix + "workflow_completed" + WorkflowCanceledCounter = TemporalMetricsPrefix + "workflow_canceled" + WorkflowFailedCounter = TemporalMetricsPrefix + "workflow_failed" + WorkflowContinueAsNewCounter = TemporalMetricsPrefix + "workflow_continue_as_new" + WorkflowEndToEndLatency = TemporalMetricsPrefix + "workflow_endtoend_latency" // measure workflow execution from start to close + + WorkflowTaskReplayLatency = TemporalMetricsPrefix + "workflow_task_replay_latency" + WorkflowTaskQueuePollEmptyCounter = TemporalMetricsPrefix + "workflow_task_queue_poll_empty" + WorkflowTaskQueuePollSucceedCounter = TemporalMetricsPrefix + "workflow_task_queue_poll_succeed" + WorkflowTaskScheduleToStartLatency = TemporalMetricsPrefix + "workflow_task_schedule_to_start_latency" + WorkflowTaskExecutionLatency = TemporalMetricsPrefix + "workflow_task_execution_latency" + WorkflowTaskExecutionFailureCounter = TemporalMetricsPrefix + "workflow_task_execution_failed" + WorkflowTaskNoCompletionCounter = TemporalMetricsPrefix + "workflow_task_no_completion" + + ActivityPollNoTaskCounter = TemporalMetricsPrefix + "activity_poll_no_task" + ActivityScheduleToStartLatency = TemporalMetricsPrefix + "activity_schedule_to_start_latency" + ActivityExecutionFailedCounter = TemporalMetricsPrefix + "activity_execution_failed" + UnregisteredActivityInvocationCounter = TemporalMetricsPrefix + "unregistered_activity_invocation" + ActivityExecutionLatency = TemporalMetricsPrefix + "activity_execution_latency" + ActivitySucceedEndToEndLatency = TemporalMetricsPrefix + "activity_succeed_endtoend_latency" + ActivityTaskErrorCounter = TemporalMetricsPrefix + "activity_task_error" + + LocalActivityTotalCounter = TemporalMetricsPrefix + "local_activity_total" + LocalActivityCanceledCounter = TemporalMetricsPrefix + "local_activity_canceled" // Deprecated: Use LocalActivityExecutionCanceledCounter instead. + LocalActivityExecutionCanceledCounter = TemporalMetricsPrefix + "local_activity_execution_cancelled" + LocalActivityFailedCounter = TemporalMetricsPrefix + "local_activity_failed" // Deprecated: Use LocalActivityExecutionFailedCounter instead. + LocalActivityExecutionFailedCounter = TemporalMetricsPrefix + "local_activity_execution_failed" + LocalActivityErrorCounter = TemporalMetricsPrefix + "local_activity_error" + LocalActivityExecutionLatency = TemporalMetricsPrefix + "local_activity_execution_latency" + LocalActivitySucceedEndToEndLatency = TemporalMetricsPrefix + "local_activity_succeed_endtoend_latency" + + CorruptedSignalsCounter = TemporalMetricsPrefix + "corrupted_signals" + + WorkerStartCounter = TemporalMetricsPrefix + "worker_start" + WorkerTaskSlotsAvailable = TemporalMetricsPrefix + "worker_task_slots_available" + WorkerTaskSlotsUsed = TemporalMetricsPrefix + "worker_task_slots_used" + PollerStartCounter = TemporalMetricsPrefix + "poller_start" + NumPoller = TemporalMetricsPrefix + "num_pollers" + + TemporalRequest = TemporalMetricsPrefix + "request" + TemporalRequestFailure = TemporalRequest + "_failure" + TemporalRequestLatency = TemporalRequest + "_latency" + TemporalLongRequest = TemporalMetricsPrefix + "long_request" + TemporalLongRequestFailure = TemporalLongRequest + "_failure" + TemporalLongRequestLatency = TemporalLongRequest + "_latency" + TemporalRequestResourceExhausted = TemporalRequest + "_resource_exhausted" + TemporalLongRequestResourceExhausted = TemporalLongRequest + "_resource_exhausted" + + StickyCacheHit = TemporalMetricsPrefix + "sticky_cache_hit" + StickyCacheMiss = TemporalMetricsPrefix + "sticky_cache_miss" + StickyCacheTotalForcedEviction = TemporalMetricsPrefix + "sticky_cache_total_forced_eviction" + StickyCacheSize = TemporalMetricsPrefix + "sticky_cache_size" + + WorkflowActiveThreadCount = TemporalMetricsPrefix + "workflow_active_thread_count" + + NexusPollNoTaskCounter = TemporalMetricsPrefix + "nexus_poll_no_task" + NexusTaskScheduleToStartLatency = TemporalMetricsPrefix + "nexus_task_schedule_to_start_latency" + NexusTaskExecutionFailedCounter = TemporalMetricsPrefix + "nexus_task_execution_failed" + NexusTaskExecutionLatency = TemporalMetricsPrefix + "nexus_task_execution_latency" + NexusTaskEndToEndLatency = TemporalMetricsPrefix + "nexus_task_endtoend_latency" +) + +// Metric tag keys +const ( + NamespaceTagName = "namespace" + ClientTagName = "client_name" + PollerTypeTagName = "poller_type" + WorkerTypeTagName = "worker_type" + WorkflowTypeNameTagName = "workflow_type" + ActivityTypeNameTagName = "activity_type" + NexusServiceTagName = "nexus_service" + NexusOperationTagName = "nexus_operation" + TaskQueueTagName = "task_queue" + OperationTagName = "operation" + CauseTagName = "cause" + WorkflowTaskFailureReason = "failure_reason" + RequestFailureCode = "status_code" +) + +// Metric tag values +const ( + NoneTagValue = "none" + ClientTagValue = "temporal_go" + PollerTypeWorkflowTask = "workflow_task" + PollerTypeWorkflowStickyTask = "workflow_sticky_task" + PollerTypeActivityTask = "activity_task" + PollerTypeNexusTask = "nexus_task" +) diff --git a/vendor/go.temporal.io/sdk/internal/common/metrics/grpc.go b/vendor/go.temporal.io/sdk/internal/common/metrics/grpc.go new file mode 100644 index 00000000000..136f95e5f1a --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/metrics/grpc.go @@ -0,0 +1,131 @@ +// The MIT License +// +// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package metrics + +import ( + "context" + "strings" + "time" + + enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/api/serviceerror" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +// HandlerContextKey is the context key for a MetricHandler value. +type HandlerContextKey struct{} + +// LongPollContextKey is the context key for a boolean stating whether the gRPC +// call is a long poll. +type LongPollContextKey struct{} + +// NewGRPCInterceptor creates a new gRPC unary interceptor to record metrics. +func NewGRPCInterceptor(defaultHandler Handler, suffix string, disableRequestFailCodes bool) grpc.UnaryClientInterceptor { + return func( + ctx context.Context, + method string, + req interface{}, + reply interface{}, + cc *grpc.ClientConn, + invoker grpc.UnaryInvoker, + opts ...grpc.CallOption, + ) error { + handler, _ := ctx.Value(HandlerContextKey{}).(Handler) + if handler == nil { + handler = defaultHandler + } + longPoll, ok := ctx.Value(LongPollContextKey{}).(bool) + if !ok { + longPoll = false + } + + // Only take method name after the last slash + operation := method[strings.LastIndex(method, "/")+1:] + + // Since this interceptor can be used for clients of different name, we + // attempt to extract the namespace out of the request. All namespace-based + // requests have been confirmed to have a top-level namespace field. + namespace := "_unknown_" + if nsReq, _ := req.(interface{ GetNamespace() string }); nsReq != nil { + namespace = nsReq.GetNamespace() + } + + // Capture time, record start, run, and record end + tags := map[string]string{OperationTagName: operation, NamespaceTagName: namespace} + handler = handler.WithTags(tags) + start := time.Now() + recordRequestStart(handler, longPoll, suffix) + err := invoker(ctx, method, req, reply, cc, opts...) + recordRequestEnd(handler, longPoll, suffix, start, err, disableRequestFailCodes) + return err + } +} + +func recordRequestStart(handler Handler, longPoll bool, suffix string) { + // Count request + metric := TemporalRequest + if longPoll { + metric = TemporalLongRequest + } + metric += suffix + handler.Counter(metric).Inc(1) +} + +func recordRequestEnd(handler Handler, longPoll bool, suffix string, start time.Time, err error, disableRequestFailCodes bool) { + // Record latency + timerMetric := TemporalRequestLatency + if longPoll { + timerMetric = TemporalLongRequestLatency + } + timerMetric += suffix + handler.Timer(timerMetric).Record(time.Since(start)) + + // Count failure + if err != nil { + failureMetric := TemporalRequestFailure + if longPoll { + failureMetric = TemporalLongRequestFailure + } + failureMetric += suffix + errStatus, _ := status.FromError(err) + if !disableRequestFailCodes { + handler = handler.WithTags(RequestFailureCodeTags(errStatus.Code())) + } + handler.Counter(failureMetric).Inc(1) + + // If it's a resource exhausted, extract cause if present and increment + if s := status.Convert(err); s.Code() == codes.ResourceExhausted { + resMetric := TemporalRequestResourceExhausted + if longPoll { + resMetric = TemporalLongRequestResourceExhausted + } + var cause enumspb.ResourceExhaustedCause + if resErr, _ := serviceerror.FromStatus(s).(*serviceerror.ResourceExhausted); resErr != nil { + cause = resErr.Cause + } + handler.WithTags(map[string]string{CauseTagName: cause.String()}).Counter(resMetric).Inc(1) + } + } +} diff --git a/vendor/go.temporal.io/sdk/internal/common/metrics/handler.go b/vendor/go.temporal.io/sdk/internal/common/metrics/handler.go new file mode 100644 index 00000000000..c2c3477d5df --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/metrics/handler.go @@ -0,0 +1,142 @@ +// The MIT License +// +// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package metrics + +import "time" + +// Handler is a handler for metrics emitted by the SDK. This interface is +// intentionally limited to only what the SDK needs to emit metrics and is not +// built to be a general purpose metrics abstraction for all uses. +// +// A common implementation is at +// go.temporal.io/sdk/contrib/tally.NewMetricsHandler. The NopHandler is a noop +// handler. A handler may implement "Unwrap() Handler" if it wraps a handler. +type Handler interface { + // WithTags returns a new handler with the given tags set for each metric + // created from it. + WithTags(map[string]string) Handler + + // Counter obtains a counter for the given name. + Counter(name string) Counter + + // Gauge obtains a gauge for the given name. + Gauge(name string) Gauge + + // Timer obtains a timer for the given name. + Timer(name string) Timer +} + +// Counter is an ever-increasing counter. +type Counter interface { + // Inc increments the counter value. + Inc(int64) +} + +// CounterFunc implements Counter with a single function. +type CounterFunc func(int64) + +// Inc implements Counter.Inc. +func (c CounterFunc) Inc(d int64) { c(d) } + +// Gauge can be set to any float. +type Gauge interface { + // Update updates the gauge value. + Update(float64) +} + +// GaugeFunc implements Gauge with a single function. +type GaugeFunc func(float64) + +// Update implements Gauge.Update. +func (g GaugeFunc) Update(d float64) { g(d) } + +// Timer records time durations. +type Timer interface { + // Record sets the timer value. + Record(time.Duration) +} + +// TimerFunc implements Timer with a single function. +type TimerFunc func(time.Duration) + +// Record implements Timer.Record. +func (t TimerFunc) Record(d time.Duration) { t(d) } + +// NopHandler is a noop handler that does nothing with the metrics. +var NopHandler Handler = nopHandler{} + +type nopHandler struct{} + +func (nopHandler) WithTags(map[string]string) Handler { return nopHandler{} } +func (nopHandler) Counter(string) Counter { return nopHandler{} } +func (nopHandler) Gauge(string) Gauge { return nopHandler{} } +func (nopHandler) Timer(string) Timer { return nopHandler{} } +func (nopHandler) Inc(int64) {} +func (nopHandler) Update(float64) {} +func (nopHandler) Record(time.Duration) {} + +type replayAwareHandler struct { + replay *bool + underlying Handler +} + +// NewReplayAwareHandler is a handler that will not record any metrics if the +// boolean pointed to by "replay" is true. +func NewReplayAwareHandler(replay *bool, underlying Handler) Handler { + return &replayAwareHandler{replay, underlying} +} + +func (r *replayAwareHandler) WithTags(tags map[string]string) Handler { + return NewReplayAwareHandler(r.replay, r.underlying.WithTags(tags)) +} + +func (r *replayAwareHandler) Counter(name string) Counter { + underlying := r.underlying.Counter(name) + return CounterFunc(func(d int64) { + if !*r.replay { + underlying.Inc(d) + } + }) +} + +func (r *replayAwareHandler) Gauge(name string) Gauge { + underlying := r.underlying.Gauge(name) + return GaugeFunc(func(d float64) { + if !*r.replay { + underlying.Update(d) + } + }) +} + +func (r *replayAwareHandler) Timer(name string) Timer { + underlying := r.underlying.Timer(name) + return TimerFunc(func(d time.Duration) { + if !*r.replay { + underlying.Record(d) + } + }) +} + +func (r *replayAwareHandler) Unwrap() Handler { + return r.underlying +} diff --git a/vendor/go.temporal.io/sdk/internal/common/metrics/tags.go b/vendor/go.temporal.io/sdk/internal/common/metrics/tags.go new file mode 100644 index 00000000000..7339af23987 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/metrics/tags.go @@ -0,0 +1,161 @@ +// The MIT License +// +// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package metrics + +import ( + "strconv" + + "google.golang.org/grpc/codes" +) + +// RootTags returns a set of base tags for all metrics. +func RootTags(namespace string) map[string]string { + return map[string]string{ + NamespaceTagName: namespace, + ClientTagName: ClientTagValue, + WorkerTypeTagName: NoneTagValue, + WorkflowTypeNameTagName: NoneTagValue, + ActivityTypeNameTagName: NoneTagValue, + TaskQueueTagName: NoneTagValue, + } +} + +// RPCTags returns a set of tags for RPC calls. +func RPCTags(workflowType, activityType, taskQueueName string) map[string]string { + return map[string]string{ + WorkflowTypeNameTagName: workflowType, + ActivityTypeNameTagName: activityType, + TaskQueueTagName: taskQueueName, + } +} + +// WorkflowTags returns a set of tags for workflows. +func WorkflowTags(workflowType string) map[string]string { + return map[string]string{ + WorkflowTypeNameTagName: workflowType, + } +} + +// ActivityTags returns a set of tags for activities. +func ActivityTags(workflowType, activityType, taskQueueName string) map[string]string { + return map[string]string{ + WorkflowTypeNameTagName: workflowType, + ActivityTypeNameTagName: activityType, + TaskQueueTagName: taskQueueName, + } +} + +// LocalActivityTags returns a set of tags for local activities. +func LocalActivityTags(workflowType, activityType string) map[string]string { + return map[string]string{ + WorkflowTypeNameTagName: workflowType, + ActivityTypeNameTagName: activityType, + } +} + +// NexusTags returns a set of tags for Nexus Operations. +func NexusTags(service, operation, taskQueueName string) map[string]string { + return map[string]string{ + NexusServiceTagName: service, + NexusOperationTagName: operation, + TaskQueueTagName: taskQueueName, + } +} + +// TaskQueueTags returns a set of tags for a task queue. +func TaskQueueTags(taskQueue string) map[string]string { + return map[string]string{ + TaskQueueTagName: taskQueue, + } +} + +// WorkerTags returns a set of tags for workers. +func WorkerTags(workerType string) map[string]string { + return map[string]string{ + WorkerTypeTagName: workerType, + } +} + +// PollerTags returns a set of tags for pollers. +func PollerTags(pollerType string) map[string]string { + return map[string]string{ + PollerTypeTagName: pollerType, + } +} + +// WorkflowTaskFailedTags returns a set of tags for a workflow task failure. +func WorkflowTaskFailedTags(reason string) map[string]string { + return map[string]string{ + WorkflowTaskFailureReason: reason, + } +} + +// RequestFailureCodeTags returns a set of tags for a request failure. +func RequestFailureCodeTags(statusCode codes.Code) map[string]string { + asStr := canonicalString(statusCode) + return map[string]string{ + RequestFailureCode: asStr, + } +} + +// Annoyingly gRPC defines this, but does not expose it publicly. +func canonicalString(c codes.Code) string { + switch c { + case codes.OK: + return "OK" + case codes.Canceled: + return "CANCELLED" + case codes.Unknown: + return "UNKNOWN" + case codes.InvalidArgument: + return "INVALID_ARGUMENT" + case codes.DeadlineExceeded: + return "DEADLINE_EXCEEDED" + case codes.NotFound: + return "NOT_FOUND" + case codes.AlreadyExists: + return "ALREADY_EXISTS" + case codes.PermissionDenied: + return "PERMISSION_DENIED" + case codes.ResourceExhausted: + return "RESOURCE_EXHAUSTED" + case codes.FailedPrecondition: + return "FAILED_PRECONDITION" + case codes.Aborted: + return "ABORTED" + case codes.OutOfRange: + return "OUT_OF_RANGE" + case codes.Unimplemented: + return "UNIMPLEMENTED" + case codes.Internal: + return "INTERNAL" + case codes.Unavailable: + return "UNAVAILABLE" + case codes.DataLoss: + return "DATA_LOSS" + case codes.Unauthenticated: + return "UNAUTHENTICATED" + default: + return "CODE(" + strconv.FormatInt(int64(c), 10) + ")" + } +} diff --git a/vendor/go.temporal.io/sdk/internal/common/retry/interceptor.go b/vendor/go.temporal.io/sdk/internal/common/retry/interceptor.go new file mode 100644 index 00000000000..0d3d1220f6a --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/retry/interceptor.go @@ -0,0 +1,180 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package retry + +import ( + "context" + "math" + "sync/atomic" + "time" + + grpc_retry "github.com/grpc-ecosystem/go-grpc-middleware/retry" + "github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +const ( + // UnlimitedMaximumAttempts when maximum attempts is set to this special value, then the number of attempts is unlimited. + UnlimitedMaximumAttempts = 0 + // UnlimitedInterval when maximum interval is set to this special value, then there is no upper bound on the retry delay. + // Should not be used together with unlimited attempts as resulting retry interval can grow to unreasonable values. + UnlimitedInterval = 0 + // DefaultBackoffCoefficient is default backOffCoefficient for retryPolicy + DefaultBackoffCoefficient = 2.0 + // DefaultMaximumInterval is default maximum amount of time for an individual retry. + DefaultMaximumInterval = 10 * time.Second + // DefaultExpirationInterval is default expiration time for all retry attempts. + DefaultExpirationInterval = time.Minute + // DefaultMaximumAttempts is default maximum number of attempts. + DefaultMaximumAttempts = UnlimitedMaximumAttempts + // DefaultJitter is a default jitter applied on the backoff interval for delay randomization. + DefaultJitter = 0.2 +) + +type ( + // GrpcRetryConfig defines required configuration for exponential backoff function that is supplied to gRPC retrier. + GrpcRetryConfig struct { + initialInterval time.Duration + backoffCoefficient float64 + maximumInterval time.Duration + expirationInterval time.Duration + jitter float64 + maximumAttempts int + } + + contextKey struct{} +) + +func (ck contextKey) String() string { + return "RetryConfig" +} + +// SetBackoffCoefficient sets rate at which backoff coefficient will change. +func (g *GrpcRetryConfig) SetBackoffCoefficient(backoffCoefficient float64) { + g.backoffCoefficient = backoffCoefficient +} + +// SetMaximumInterval defines maximum amount of time between attempts. +func (g *GrpcRetryConfig) SetMaximumInterval(maximumInterval time.Duration) { + g.maximumInterval = maximumInterval +} + +// SetExpirationInterval defines total amount of time that can be used for all retry attempts. +// Note that this value is ignored if deadline is set on the context. +func (g *GrpcRetryConfig) SetExpirationInterval(expirationInterval time.Duration) { + g.expirationInterval = expirationInterval +} + +// SetJitter defines level of randomization for each delay interval. For example 0.2 would mex target +- 20% +func (g *GrpcRetryConfig) SetJitter(jitter float64) { + g.jitter = jitter +} + +// SetMaximumAttempts defines maximum total number of retry attempts. +func (g *GrpcRetryConfig) SetMaximumAttempts(maximumAttempts int) { + g.maximumAttempts = maximumAttempts +} + +// NewGrpcRetryConfig creates new retry config with specified initial interval and defaults for other parameters. +// Use SetXXX functions on this config in order to customize values. +func NewGrpcRetryConfig(initialInterval time.Duration) *GrpcRetryConfig { + return &GrpcRetryConfig{ + initialInterval: initialInterval, + backoffCoefficient: DefaultBackoffCoefficient, + maximumInterval: DefaultMaximumInterval, + expirationInterval: DefaultExpirationInterval, + jitter: DefaultJitter, + maximumAttempts: DefaultMaximumAttempts, + } +} + +var ( + // ConfigKey context key for GrpcRetryConfig + ConfigKey = contextKey{} + // gRPC response codes that represent retryable errors. + // The following status codes are never generated by the library: + // INVALID_ARGUMENT, NOT_FOUND, ALREADY_EXISTS, FAILED_PRECONDITION, ABORTED, OUT_OF_RANGE, DATA_LOSS + // codes.DeadlineExceeded and codes.Canceled are not here (and shouldn't be here!) + // because they are coming from go context and "context errors are not retriable based on user settings" + // by gRPC library. + retryableCodes = []codes.Code{codes.Aborted, codes.Internal, + codes.ResourceExhausted, codes.Unavailable, codes.Unknown} + retryableCodesWithoutInternal = []codes.Code{codes.Aborted, + codes.ResourceExhausted, codes.Unavailable, codes.Unknown} +) + +// NewRetryOptionsInterceptor creates a new gRPC interceptor that populates retry options for each call based on values +// provided in the context. The atomic bool is checked each call to determine whether internals are included in retry. +// If not present or false, internals are assumed to be included. +func NewRetryOptionsInterceptor(excludeInternal *atomic.Bool) grpc.UnaryClientInterceptor { + return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + if rc, ok := ctx.Value(ConfigKey).(*GrpcRetryConfig); ok { + if _, ok := ctx.Deadline(); !ok { + deadlineCtx, cancel := context.WithDeadline(ctx, time.Now().Add(rc.expirationInterval)) + defer cancel() + ctx = deadlineCtx + } + // Populate backoff function, which provides retrier with the delay for each attempt. + opts = append(opts, grpc_retry.WithBackoff(func(attempt uint) time.Duration { + next := float64(rc.initialInterval) * math.Pow(rc.backoffCoefficient, float64(attempt)) + if rc.maximumInterval != UnlimitedInterval { + next = math.Min(next, float64(rc.maximumInterval)) + } + return backoffutils.JitterUp(time.Duration(next), rc.jitter) + })) + // Max attempts is a required parameter in grpc retry interceptor, + // if it's set to zero then no retries will be made. + if rc.maximumAttempts != UnlimitedMaximumAttempts { + opts = append(opts, grpc_retry.WithMax(uint(rc.maximumAttempts))) + } else { + opts = append(opts, grpc_retry.WithMax(math.MaxUint32)) + } + // We have to deal with plain gRPC error codes instead of service errors here as actual error translation + // happens after invoker is called below and invoker must have correct retry options right away in order to + // supply them to the gRPC retrier. + if excludeInternal != nil && excludeInternal.Load() { + opts = append(opts, grpc_retry.WithCodes(retryableCodesWithoutInternal...)) + } else { + opts = append(opts, grpc_retry.WithCodes(retryableCodes...)) + } + } else { + // Do not retry if retry config is not set. + opts = append(opts, grpc_retry.Disable()) + } + return invoker(ctx, method, req, reply, cc, opts...) + } +} + +// IsStatusCodeRetryable returns true if error code in the status is retryable. +func IsStatusCodeRetryable(status *status.Status) bool { + for _, retryable := range retryableCodes { + if retryable == status.Code() { + return true + } + } + return false +} diff --git a/vendor/go.temporal.io/sdk/internal/common/serializer/jsonpb.go b/vendor/go.temporal.io/sdk/internal/common/serializer/jsonpb.go new file mode 100644 index 00000000000..dd273f52c5f --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/serializer/jsonpb.go @@ -0,0 +1,61 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serializer + +import ( + "google.golang.org/protobuf/encoding/protojson" + "google.golang.org/protobuf/proto" +) + +type ( + // JSONPBEncoder is JSON encoder/decoder for protobuf structs and slices of protobuf structs. + JSONPBEncoder struct { + opts protojson.MarshalOptions + } +) + +// NewJSONPBEncoder creates a new JSONPBEncoder. +func NewJSONPBEncoder() JSONPBEncoder { + return JSONPBEncoder{} +} + +// NewJSONPBIndentEncoder creates a new JSONPBEncoder with indent. +func NewJSONPBIndentEncoder(indent string) JSONPBEncoder { + return JSONPBEncoder{ + opts: protojson.MarshalOptions{ + Indent: indent, + }, + } +} + +// Encode protobuf struct to bytes. +func (e JSONPBEncoder) Encode(pb proto.Message) ([]byte, error) { + return e.opts.Marshal(pb) +} + +// Decode bytes to protobuf struct. +func (e JSONPBEncoder) Decode(data []byte, pb proto.Message) error { + return protojson.Unmarshal(data, pb) +} diff --git a/vendor/go.temporal.io/sdk/internal/common/serializer/serializer.go b/vendor/go.temporal.io/sdk/internal/common/serializer/serializer.go new file mode 100644 index 00000000000..5ee3656fcff --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/serializer/serializer.go @@ -0,0 +1,215 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package serializer + +import ( + "encoding/json" + "fmt" + + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + historypb "go.temporal.io/api/history/v1" + "go.temporal.io/api/serviceerror" + "google.golang.org/protobuf/proto" +) + +type ( + + // SerializationError is an error type for serialization + SerializationError struct { + msg string + } + + // DeserializationError is an error type for deserialization + DeserializationError struct { + msg string + } + + // UnknownEncodingTypeError is an error type for unknown or unsupported encoding type + UnknownEncodingTypeError struct { + encodingType enumspb.EncodingType + } + + // Marshaler is implemented by objects that can marshal themselves + Marshaler interface { + Marshal() ([]byte, error) + } +) + +// SerializeBatchEvents serializes batch events into a datablob proto +func SerializeBatchEvents(events []*historypb.HistoryEvent, encodingType enumspb.EncodingType) (*commonpb.DataBlob, error) { + return serialize(&historypb.History{Events: events}, encodingType) +} + +func serializeProto(p Marshaler, encodingType enumspb.EncodingType) (*commonpb.DataBlob, error) { + if p == nil { + return nil, nil + } + + var data []byte + var err error + + switch encodingType { + case enumspb.ENCODING_TYPE_PROTO3: + data, err = p.Marshal() + case enumspb.ENCODING_TYPE_JSON: + encodingType = enumspb.ENCODING_TYPE_JSON + pb, ok := p.(proto.Message) + if !ok { + return nil, NewSerializationError("could not cast protomarshal interface to proto.message") + } + data, err = NewJSONPBEncoder().Encode(pb) + default: + return nil, NewUnknownEncodingTypeError(encodingType) + } + + if err != nil { + return nil, NewSerializationError(err.Error()) + } + + // Shouldn't happen, but keeping + if data == nil { + return nil, nil + } + + return NewDataBlob(data, encodingType), nil +} + +// DeserializeBatchEvents deserializes batch events from a datablob proto +func DeserializeBatchEvents(data *commonpb.DataBlob) ([]*historypb.HistoryEvent, error) { + if data == nil { + return nil, nil + } + if len(data.Data) == 0 { + return nil, nil + } + + events := &historypb.History{} + var err error + switch data.EncodingType { + case enumspb.ENCODING_TYPE_JSON: + err = NewJSONPBEncoder().Decode(data.Data, events) + case enumspb.ENCODING_TYPE_PROTO3: + err = proto.Unmarshal(data.Data, events) + default: + return nil, NewDeserializationError("DeserializeBatchEvents invalid encoding") + } + if err != nil { + return nil, err + } + return events.Events, nil +} + +func serialize(input interface{}, encodingType enumspb.EncodingType) (*commonpb.DataBlob, error) { + if input == nil { + return nil, nil + } + + if p, ok := input.(Marshaler); ok { + return serializeProto(p, encodingType) + } + + var data []byte + var err error + + switch encodingType { + case enumspb.ENCODING_TYPE_JSON: // For backward-compatibility + data, err = json.Marshal(input) + default: + return nil, NewUnknownEncodingTypeError(encodingType) + } + + if err != nil { + return nil, NewSerializationError(err.Error()) + } + + return NewDataBlob(data, encodingType), nil +} + +// NewUnknownEncodingTypeError returns a new instance of encoding type error +func NewUnknownEncodingTypeError(encodingType enumspb.EncodingType) error { + return &UnknownEncodingTypeError{encodingType: encodingType} +} + +func (e *UnknownEncodingTypeError) Error() string { + return fmt.Sprintf("unknown or unsupported encoding type %v", e.encodingType) +} + +// NewSerializationError returns a SerializationError +func NewSerializationError(msg string) error { + return &SerializationError{msg: msg} +} + +func (e *SerializationError) Error() string { + return fmt.Sprintf("serialization error: %v", e.msg) +} + +// NewDeserializationError returns a DeserializationError +func NewDeserializationError(msg string) error { + return &DeserializationError{msg: msg} +} + +func (e *DeserializationError) Error() string { + return fmt.Sprintf("deserialization error: %v", e.msg) +} + +// NewDataBlob creates new blob data +func NewDataBlob(data []byte, encodingType enumspb.EncodingType) *commonpb.DataBlob { + if len(data) == 0 { + return nil + } + + return &commonpb.DataBlob{ + Data: data, + EncodingType: encodingType, + } +} + +// DeserializeBlobDataToHistoryEvents deserialize the blob data to history event data +func DeserializeBlobDataToHistoryEvents( + dataBlobs []*commonpb.DataBlob, filterType enumspb.HistoryEventFilterType, +) (*historypb.History, error) { + + var historyEvents []*historypb.HistoryEvent + + for _, batch := range dataBlobs { + events, err := DeserializeBatchEvents(batch) + if err != nil { + return nil, err + } + if len(events) == 0 { + return nil, &serviceerror.Internal{ + Message: "corrupted history event batch, empty events", + } + } + + historyEvents = append(historyEvents, events...) + } + + if filterType == enumspb.HISTORY_EVENT_FILTER_TYPE_CLOSE_EVENT { + historyEvents = []*historypb.HistoryEvent{historyEvents[len(historyEvents)-1]} + } + return &historypb.History{Events: historyEvents}, nil +} diff --git a/vendor/go.temporal.io/sdk/internal/common/util/once_cell.go b/vendor/go.temporal.io/sdk/internal/common/util/once_cell.go new file mode 100644 index 00000000000..0027ec8675d --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/util/once_cell.go @@ -0,0 +1,69 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package util + +import "sync" + +// A OnceCell attempts to match the semantics of Rust's `OnceCell`, but only stores strings, since that's what's needed +// at the moment. Could be changed to use interface{} to be generic. +type OnceCell struct { + // Ensures we only call the fetcher one time + once sync.Once + // Stores the result of calling fetcher + value string + fetcher func() string +} + +// Get fetches the value in the cell, calling the fetcher function if it has not yet been called +func (oc *OnceCell) Get() string { + oc.once.Do(func() { + res := oc.fetcher() + oc.value = res + }) + return oc.value +} + +// PopulatedOnceCell creates an already-initialized cell +func PopulatedOnceCell(value string) OnceCell { + return OnceCell{ + once: sync.Once{}, + value: value, + fetcher: func() string { + return value + }, + } +} + +type fetcher func() string + +// LazyOnceCell creates a cell with no initial value, the provided function will be called once and only once the first +// time OnceCell.Get is called +func LazyOnceCell(fetcher fetcher) OnceCell { + return OnceCell{ + once: sync.Once{}, + value: "", + fetcher: fetcher, + } +} diff --git a/vendor/go.temporal.io/sdk/internal/common/util/stringer.go b/vendor/go.temporal.io/sdk/internal/common/util/stringer.go new file mode 100644 index 00000000000..307730bfbb5 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/util/stringer.go @@ -0,0 +1,195 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package util + +import ( + "bytes" + "fmt" + "reflect" + "regexp" + + commandpb "go.temporal.io/api/command/v1" + enumspb "go.temporal.io/api/enums/v1" + historypb "go.temporal.io/api/history/v1" +) + +var privateField = regexp.MustCompile("^[a-z]") + +func anyToString(d interface{}) string { + v := reflect.ValueOf(d) + switch v.Kind() { + case reflect.Ptr: + return anyToString(v.Elem().Interface()) + case reflect.Struct: + var buf bytes.Buffer + t := reflect.TypeOf(d) + buf.WriteString("(") + for i := 0; i < v.NumField(); i++ { + f := v.Field(i) + if f.Kind() == reflect.Invalid || privateField.MatchString(t.Field(i).Name) { + continue + } + fieldValue := valueToString(f) + if len(fieldValue) == 0 { + continue + } + if buf.Len() > 1 { + buf.WriteString(", ") + } + buf.WriteString(fmt.Sprintf("%s:%s", t.Field(i).Name, fieldValue)) + } + buf.WriteString(")") + return buf.String() + default: + return fmt.Sprint(d) + } +} + +func valueToString(v reflect.Value) string { + switch v.Kind() { + case reflect.Ptr: + return valueToString(v.Elem()) + case reflect.Struct: + if v.CanInterface() { + return anyToString(v.Interface()) + } + case reflect.Invalid: + return "" + case reflect.Slice: + if v.Type().Elem().Kind() == reflect.Uint8 { + return fmt.Sprintf("[%v]", string(v.Bytes())) + } + return fmt.Sprintf("[len=%d]", v.Len()) + default: + if v.CanInterface() { + return fmt.Sprint(v.Interface()) + } + } + + return "" +} + +// HistoryEventToString convert HistoryEvent to string +func HistoryEventToString(e *historypb.HistoryEvent) string { + var data interface{} + switch e.GetEventType() { + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED: + data = e.GetWorkflowExecutionStartedEventAttributes() + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED: + data = e.GetWorkflowExecutionCompletedEventAttributes() + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_FAILED: + data = e.GetWorkflowExecutionFailedEventAttributes() + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_TIMED_OUT: + data = e.GetWorkflowExecutionTimedOutEventAttributes() + + case enumspb.EVENT_TYPE_WORKFLOW_TASK_SCHEDULED: + data = e.GetWorkflowTaskScheduledEventAttributes() + + case enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED: + data = e.GetWorkflowTaskStartedEventAttributes() + + case enumspb.EVENT_TYPE_WORKFLOW_TASK_COMPLETED: + data = e.GetWorkflowTaskCompletedEventAttributes() + + case enumspb.EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT: + data = e.GetWorkflowTaskTimedOutEventAttributes() + + case enumspb.EVENT_TYPE_ACTIVITY_TASK_SCHEDULED: + data = e.GetActivityTaskScheduledEventAttributes() + + case enumspb.EVENT_TYPE_ACTIVITY_TASK_STARTED: + data = e.GetActivityTaskStartedEventAttributes() + + case enumspb.EVENT_TYPE_ACTIVITY_TASK_COMPLETED: + data = e.GetActivityTaskCompletedEventAttributes() + + case enumspb.EVENT_TYPE_ACTIVITY_TASK_FAILED: + data = e.GetActivityTaskFailedEventAttributes() + + case enumspb.EVENT_TYPE_ACTIVITY_TASK_TIMED_OUT: + data = e.GetActivityTaskTimedOutEventAttributes() + + case enumspb.EVENT_TYPE_ACTIVITY_TASK_CANCEL_REQUESTED: + data = e.GetActivityTaskCancelRequestedEventAttributes() + + case enumspb.EVENT_TYPE_ACTIVITY_TASK_CANCELED: + data = e.GetActivityTaskCanceledEventAttributes() + + case enumspb.EVENT_TYPE_TIMER_STARTED: + data = e.GetTimerStartedEventAttributes() + + case enumspb.EVENT_TYPE_TIMER_FIRED: + data = e.GetTimerFiredEventAttributes() + + case enumspb.EVENT_TYPE_TIMER_CANCELED: + data = e.GetTimerCanceledEventAttributes() + + case enumspb.EVENT_TYPE_MARKER_RECORDED: + data = e.GetMarkerRecordedEventAttributes() + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_TERMINATED: + data = e.GetWorkflowExecutionTerminatedEventAttributes() + + default: + data = e + } + + return e.GetEventType().String() + ": " + anyToString(data) +} + +// CommandToString convert Command to string +func CommandToString(d *commandpb.Command) string { + var data interface{} + switch d.GetCommandType() { + case enumspb.COMMAND_TYPE_SCHEDULE_ACTIVITY_TASK: + data = d.GetScheduleActivityTaskCommandAttributes() + + case enumspb.COMMAND_TYPE_REQUEST_CANCEL_ACTIVITY_TASK: + data = d.GetRequestCancelActivityTaskCommandAttributes() + + case enumspb.COMMAND_TYPE_START_TIMER: + data = d.GetStartTimerCommandAttributes() + + case enumspb.COMMAND_TYPE_CANCEL_TIMER: + data = d.GetCancelTimerCommandAttributes() + + case enumspb.COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION: + data = d.GetCompleteWorkflowExecutionCommandAttributes() + + case enumspb.COMMAND_TYPE_FAIL_WORKFLOW_EXECUTION: + data = d.GetFailWorkflowExecutionCommandAttributes() + + case enumspb.COMMAND_TYPE_RECORD_MARKER: + data = d.GetRecordMarkerCommandAttributes() + + default: + data = d + } + + return d.GetCommandType().String() + ": " + anyToString(data) +} diff --git a/vendor/go.temporal.io/sdk/internal/common/util/util.go b/vendor/go.temporal.io/sdk/internal/common/util/util.go new file mode 100644 index 00000000000..31dc0448b6f --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/common/util/util.go @@ -0,0 +1,75 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package util + +import ( + "reflect" + "sync" + "time" +) + +// MergeDictoRight copies the contents of src to dest +func MergeDictoRight(src map[string]string, dest map[string]string) { + for k, v := range src { + dest[k] = v + } +} + +// MergeDicts creates a union of the two dicts +func MergeDicts(dic1 map[string]string, dic2 map[string]string) (resultDict map[string]string) { + resultDict = make(map[string]string) + MergeDictoRight(dic1, resultDict) + MergeDictoRight(dic2, resultDict) + return +} + +// AwaitWaitGroup calls Wait on the given wait +// Returns true if the Wait() call succeeded before the timeout +// Returns false if the Wait() did not return before the timeout +func AwaitWaitGroup(wg *sync.WaitGroup, timeout time.Duration) bool { + + doneC := make(chan struct{}) + + go func() { + wg.Wait() + close(doneC) + }() + + timer := time.NewTimer(timeout) + defer func() { timer.Stop() }() + + select { + case <-doneC: + return true + case <-timer.C: + return false + } +} + +// IsInterfaceNil check if interface is nil +func IsInterfaceNil(i interface{}) bool { + v := reflect.ValueOf(i) + return i == nil || (v.Kind() == reflect.Ptr && v.IsNil()) +} diff --git a/vendor/go.temporal.io/sdk/internal/context.go b/vendor/go.temporal.io/sdk/internal/context.go new file mode 100644 index 00000000000..32dde60b900 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/context.go @@ -0,0 +1,364 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "fmt" + "sync" + "time" + + enumspb "go.temporal.io/api/enums/v1" +) + +// Context is a clone of context.Context with Done() returning Channel instead +// of native channel. +// A Context carries a deadline, a cancellation signal, and other values across +// API boundaries. +// +// Context's methods may be called by multiple goroutines simultaneously. +type Context interface { + // Deadline returns the time when work done on behalf of this context + // should be canceled. Deadline returns ok==false when no deadline is + // set. Successive calls to Deadline return the same results. + Deadline() (deadline time.Time, ok bool) + + // Done returns a channel that's closed when work done on behalf of this + // context should be canceled. Done may return nil if this context can + // never be canceled. Successive calls to Done return the same value. + // + // WithCancel arranges for Done to be closed when cancel is called; + // WithDeadline arranges for Done to be closed when the deadline + // expires; WithTimeout arranges for Done to be closed when the timeout + // elapses. + // + // Done is provided for use in select statements: + // + // // Stream generates values with DoSomething and sends them to out + // // until DoSomething returns an error or ctx.Done is closed. + // func Stream(ctx Context, out Channel) (err error) { + // for { + // v, err := DoSomething(ctx) + // if err != nil { + // return err + // } + // s := NewSelector(ctx) + // s.AddReceive(ctx.Done(), func(v interface{}) { err = ctx.Err() }) + // s.AddReceive(v, func(v interface{}, more bool) { out.Send(ctx, v) }) + // s.Select(ctx) + // if err != nil { + // return err + // } + // } + // } + // + // See http://blog.golang.org/pipelines for more examples of how to use + // a Done channel for cancellation. + Done() Channel + + // Err returns a non-nil error value after Done is closed. Err returns + // Canceled if the context was canceled or DeadlineExceeded if the + // context's deadline passed. No other values for Err are defined. + // After Done is closed, successive calls to Err return the same value. + Err() error + + // Value returns the value associated with this context for key, or nil + // if no value is associated with key. Successive calls to Value with + // the same key returns the same result. + // + // Use context values only for request-scoped data that transits + // processes and API boundaries, not for passing optional parameters to + // functions. + // + // A key identifies a specific value in a Context. Functions that wish + // to store values in Context typically allocate a key in a global + // variable then use that key as the argument to context.WithValue and + // Context.Value. A key can be any type that supports equality; + // packages should define keys as an unexported type to avoid + // collisions. + // + // Packages that define a Context key should provide type-safe accessors + // for the values stores using that key: + // + // // Package user defines a User type that's stored in Contexts. + // package user + // + // import "context" + // + // // User is the type of value stored in the Contexts. + // type User struct {...} + // + // // key is an unexported type for keys defined in this package. + // // This prevents collisions with keys defined in other packages. + // type key int + // + // // userKey is the key for user.User values in Contexts. It is + // // unexported; clients use user.NewContext and user.FromContext + // // instead of using this key directly. + // var userKey key = 0 + // + // // NewContext returns a new Context that carries value u. + // func NewContext(ctx context.Context, u *User) context.Context { + // return context.WithValue(ctx, userKey, u) + // } + // + // // FromContext returns the User value stored in ctx, if any. + // func FromContext(ctx context.Context) (*User, bool) { + // u, ok := ctx.Value(userKey).(*User) + // return u, ok + // } + Value(key interface{}) interface{} +} + +// An emptyCtx is never canceled, has no values, and has no deadline. It is not +// struct{}, since vars of this type must have distinct addresses. +type emptyCtx int + +func (*emptyCtx) Deadline() (deadline time.Time, ok bool) { + return +} + +func (*emptyCtx) Done() Channel { + return nil +} + +func (*emptyCtx) Err() error { + return nil +} + +func (*emptyCtx) Value(_ interface{}) interface{} { + return nil +} + +func (e *emptyCtx) String() string { + switch e { + case background: + return "context.Background" + case todo: + return "context.TODO" + } + return "unknown empty Context" +} + +var ( + background = new(emptyCtx) + todo = new(emptyCtx) +) + +// Background returns a non-nil, empty Context. It is never canceled, has no +// values, and has no deadline +func Background() Context { + return background +} + +// ErrCanceled is the error returned by Context.Err when the context is canceled. +var ErrCanceled = NewCanceledError() + +// ErrDeadlineExceeded is the error returned by Context.Err when the context's +// deadline passes. +var ErrDeadlineExceeded = NewTimeoutError("deadline exceeded", enumspb.TIMEOUT_TYPE_SCHEDULE_TO_CLOSE, nil) + +// A CancelFunc tells an operation to abandon its work. +// A CancelFunc does not wait for the work to stop. +// After the first call, subsequent calls to a CancelFunc do nothing. +type CancelFunc func() + +// WithCancel returns a copy of parent with a new Done channel. The returned +// context's Done channel is closed when the returned cancel function is called +// or when the parent context's Done channel is closed, whichever happens first. +// +// Canceling this context releases resources associated with it, so code should +// call cancel as soon as the operations running in this Context complete. +func WithCancel(parent Context) (ctx Context, cancel CancelFunc) { + c := newCancelCtx(parent) + propagateCancel(parent, c) + return c, func() { c.cancel(true, ErrCanceled) } +} + +// NewDisconnectedContext returns a new context that won't propagate parent's cancellation to the new child context. +// One common use case is to do cleanup work after workflow is canceled. +// +// err := workflow.ExecuteActivity(ctx, ActivityFoo).Get(ctx, &activityFooResult) +// if err != nil && temporal.IsCanceledError(ctx.Err()) { +// // activity failed, and workflow context is canceled +// disconnectedCtx, _ := workflow.NewDisconnectedContext(ctx); +// workflow.ExecuteActivity(disconnectedCtx, handleCancellationActivity).Get(disconnectedCtx, nil) +// return err // workflow return CanceledError +// } +func NewDisconnectedContext(parent Context) (ctx Context, cancel CancelFunc) { + c := newCancelCtx(parent) + return c, func() { c.cancel(true, ErrCanceled) } +} + +// newCancelCtx returns an initialized cancelCtx. +func newCancelCtx(parent Context) *cancelCtx { + return &cancelCtx{ + Context: parent, + done: NewNamedChannel(parent, "cancelCtx-done-channel"), + } +} + +// propagateCancel arranges for child to be canceled when parent is. +func propagateCancel(parent Context, child canceler) { + if parent.Done() == nil { + return // parent is never canceled + } + if p, ok := parentCancelCtx(parent); ok { + if parentErr := p.Err(); parentErr != nil { + // parent has already been canceled + child.cancel(false, parentErr) + } else { + p.childrenLock.Lock() + if p.children == nil { + p.children = make(map[canceler]bool) + } + p.children[child] = true + p.childrenLock.Unlock() + } + } else { + panic("cancelCtx not found") + } +} + +// parentCancelCtx follows a chain of parent references until it finds a +// *cancelCtx. This function understands how each of the concrete types in this +// package represents its parent. +func parentCancelCtx(parent Context) (*cancelCtx, bool) { + for { + switch c := parent.(type) { + case *cancelCtx: + return c, true + case *valueCtx: + parent = c.Context + default: + return nil, false + } + } +} + +// removeChild removes a context from its parent. +func removeChild(parent Context, child canceler) { + p, ok := parentCancelCtx(parent) + if !ok { + return + } + p.childrenLock.Lock() + if p.children != nil { + delete(p.children, child) + } + p.childrenLock.Unlock() +} + +// A canceler is a context type that can be canceled directly. The +// implementations are *cancelCtx and *timerCtx. +type canceler interface { + cancel(removeFromParent bool, err error) + Done() Channel +} + +// A cancelCtx can be canceled. When canceled, it also cancels any children +// that implement canceler. +type cancelCtx struct { + Context + + done Channel // closed by the first cancel call. + + children map[canceler]bool // set to nil by the first cancel call + childrenLock sync.Mutex + err error // set to non-nil by the first cancel call + errLock sync.RWMutex +} + +func (c *cancelCtx) Done() Channel { + return c.done +} + +func (c *cancelCtx) Err() error { + c.errLock.RLock() + defer c.errLock.RUnlock() + return c.err +} + +func (c *cancelCtx) String() string { + return fmt.Sprintf("%v.WithCancel", c.Context) +} + +// cancel closes c.done, cancels each of c's children, and, if +// removeFromParent is true, removes c from its parent's children. +func (c *cancelCtx) cancel(removeFromParent bool, err error) { + if err == nil { + panic("context: internal error: missing cancel error") + } + // This can be called from separate goroutines concurrently, so we use the + // presence of the error under lock to prevent duplicate calls + c.errLock.Lock() + alreadyCancelled := c.err != nil + if !alreadyCancelled { + c.err = err + } + c.errLock.Unlock() + if alreadyCancelled { + return + } + c.done.Close() + c.childrenLock.Lock() + children := c.children + c.children = nil + c.childrenLock.Unlock() + for child := range children { + // NOTE: acquiring the child's lock while holding parent's lock. + child.cancel(false, err) + } + + if removeFromParent { + removeChild(c.Context, c) + } +} + +// WithValue returns a copy of parent in which the value associated with key is +// val. +// +// Use context Values only for request-scoped data that transits processes and +// APIs, not for passing optional parameters to functions. +func WithValue(parent Context, key interface{}, val interface{}) Context { + return &valueCtx{parent, key, val} +} + +// A valueCtx carries a key-value pair. It implements Value for that key and +// delegates all other calls to the embedded Context. +type valueCtx struct { + Context + key, val interface{} +} + +func (c *valueCtx) String() string { + return fmt.Sprintf("%v.WithValue(%#v, %#v)", c.Context, c.key, c.val) +} + +func (c *valueCtx) Value(key interface{}) interface{} { + if c.key == key { + return c.val + } + return c.Context.Value(key) +} diff --git a/vendor/go.temporal.io/sdk/internal/encode_args.go b/vendor/go.temporal.io/sdk/internal/encode_args.go new file mode 100644 index 00000000000..f2999c18c45 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/encode_args.go @@ -0,0 +1,138 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "errors" + "fmt" + "reflect" + + commonpb "go.temporal.io/api/common/v1" + + "go.temporal.io/sdk/converter" +) + +// encode multiple arguments(arguments to a function). +func encodeArgs(dc converter.DataConverter, args []interface{}) (*commonpb.Payloads, error) { + return dc.ToPayloads(args...) +} + +// decode multiple arguments(arguments to a function). +func decodeArgs(dc converter.DataConverter, fnType reflect.Type, data *commonpb.Payloads) (result []reflect.Value, err error) { + r, err := decodeArgsToPointerValues(dc, fnType, data) + if err != nil { + return + } + for i := 0; i < len(r); i++ { + result = append(result, reflect.ValueOf(r[i]).Elem()) + } + return +} + +func decodeArgsToPointerValues(dc converter.DataConverter, fnType reflect.Type, data *commonpb.Payloads) (result []interface{}, err error) { +argsLoop: + for i := 0; i < fnType.NumIn(); i++ { + argT := fnType.In(i) + if i == 0 && (isActivityContext(argT) || isWorkflowContext(argT)) { + continue argsLoop + } + arg := reflect.New(argT).Interface() + result = append(result, arg) + } + err = dc.FromPayloads(data, result...) + if err != nil { + return + } + return +} + +func decodeArgsToRawValues(dc converter.DataConverter, fnType reflect.Type, data *commonpb.Payloads) ([]interface{}, error) { + // Build pointers to results + var pointers []interface{} + for i := 0; i < fnType.NumIn(); i++ { + argT := fnType.In(i) + if i == 0 && (isActivityContext(argT) || isWorkflowContext(argT)) { + continue + } + pointers = append(pointers, reflect.New(argT).Interface()) + } + + // Unmarshal + if err := dc.FromPayloads(data, pointers...); err != nil { + return nil, err + } + + // Convert results back to non-pointer versions + results := make([]interface{}, len(pointers)) + for i, pointer := range pointers { + result := reflect.ValueOf(pointer).Elem() + // Do not set nil pointers + if result.Kind() != reflect.Ptr || !result.IsNil() { + results[i] = result.Interface() + } + } + + return results, nil +} + +// encode single value(like return parameter). +func encodeArg(dc converter.DataConverter, arg interface{}) (*commonpb.Payloads, error) { + return dc.ToPayloads(arg) +} + +// decode single value(like return parameter). +func decodeArg(dc converter.DataConverter, data *commonpb.Payloads, valuePtr interface{}) error { + return dc.FromPayloads(data, valuePtr) +} + +func decodeAndAssignValue(dc converter.DataConverter, from interface{}, toValuePtr interface{}) error { + if toValuePtr == nil { + return nil + } + if rf := reflect.ValueOf(toValuePtr); rf.Type().Kind() != reflect.Ptr { + return errors.New("value parameter provided is not a pointer") + } + if data, ok := from.(*commonpb.Payloads); ok { + if err := decodeArg(dc, data, toValuePtr); err != nil { + return err + } + } else if fv := reflect.ValueOf(from); fv.IsValid() { + fromType := fv.Type() + toType := reflect.TypeOf(toValuePtr).Elem() + // If the value set was a pointer and is the same type as the wanted result, + // instead of panicking because it is not a pointer to a pointer, we will + // just set the pointer + if fv.Kind() == reflect.Ptr && fromType.Elem() == toType { + reflect.ValueOf(toValuePtr).Elem().Set(fv.Elem()) + } else { + assignable := fromType.AssignableTo(toType) + if !assignable { + return fmt.Errorf("%s is not assignable to %s", fromType.Name(), toType.Name()) + } + reflect.ValueOf(toValuePtr).Elem().Set(fv) + } + } + return nil +} diff --git a/vendor/go.temporal.io/sdk/internal/error.go b/vendor/go.temporal.io/sdk/internal/error.go new file mode 100644 index 00000000000..9dde77c49cc --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/error.go @@ -0,0 +1,945 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "errors" + "fmt" + "reflect" + "strings" + "time" + + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + failurepb "go.temporal.io/api/failure/v1" + + "go.temporal.io/sdk/converter" +) + +/* +If activity fails then *ActivityError is returned to the workflow code. The error has important information about activity +and actual error which caused activity failure. This internal error can be unwrapped using errors.Unwrap() or checked using errors.As(). +Below are the possible types of internal error: +1) *ApplicationError: (this should be the most common one) + *ApplicationError can be returned in two cases: + - If activity implementation returns *ApplicationError by using NewApplicationError()/NewNonRetryableApplicationError() API. + The error would contain a message and optional details. Workflow code could extract details to string typed variable, determine + what kind of error it was, and take actions based on it. The details are encoded payload therefore, workflow code needs to know what + the types of the encoded details are before extracting them. + - If activity implementation returns errors other than from NewApplicationError() API. In this case GetOriginalType() + will return original type of error represented as string. Workflow code could check this type to determine what kind of error it was + and take actions based on the type. These errors are retryable by default, unless error type is specified in retry policy. +2) *CanceledError: + If activity was canceled, internal error will be an instance of *CanceledError. When activity cancels itself by + returning NewCancelError() it would supply optional details which could be extracted by workflow code. +3) *TimeoutError: + If activity was timed out (several timeout types), internal error will be an instance of *TimeoutError. The err contains + details about what type of timeout it was. +4) *PanicError: + If activity code panic while executing, temporal activity worker will report it as activity failure to temporal server. + The SDK will present that failure as *PanicError. The error contains a string representation of the panic message and + the call stack when panic was happen. +Workflow code could handle errors based on different types of error. Below is sample code of how error handling looks like. + +err := workflow.ExecuteActivity(ctx, MyActivity, ...).Get(ctx, nil) +if err != nil { + var applicationErr *ApplicationError + if errors.As(err, &applicationError) { + // retrieve error message + fmt.Println(applicationError.Error()) + + // handle activity errors (created via NewApplicationError() API) + var detailMsg string // assuming activity return error by NewApplicationError("message", true, "string details") + applicationErr.Details(&detailMsg) // extract strong typed details + + // handle activity errors (errors created other than using NewApplicationError() API) + switch err.Type() { + case "CustomErrTypeA": + // handle CustomErrTypeA + case CustomErrTypeB: + // handle CustomErrTypeB + default: + // newer version of activity could return new errors that workflow was not aware of. + } + } + + var canceledErr *CanceledError + if errors.As(err, &canceledErr) { + // handle cancellation + } + + var timeoutErr *TimeoutError + if errors.As(err, &timeoutErr) { + // handle timeout, could check timeout type by timeoutErr.TimeoutType() + switch err.TimeoutType() { + case enumspb.TIMEOUT_TYPE_SCHEDULE_TO_START: + // Handle ScheduleToStart timeout. + case enumspb.TIMEOUT_TYPE_START_TO_CLOSE: + // Handle StartToClose timeout. + case enumspb.TIMEOUT_TYPE_HEARTBEAT: + // Handle heartbeat timeout. + default: + } + } + + var panicErr *PanicError + if errors.As(err, &panicErr) { + // handle panic, message and stack trace are available by panicErr.Error() and panicErr.StackTrace() + } +} +Errors from child workflow should be handled in a similar way, except that instance of *ChildWorkflowExecutionError is returned to +workflow code. It might contain *ActivityError in case if error comes from activity (which in turn will contain on of the errors above), +or *ApplicationError in case if error comes from child workflow itself. + +When panic happen in workflow implementation code, SDK catches that panic and causing the workflow task timeout. +That workflow task will be retried at a later time (with exponential backoff retry intervals). +Workflow consumers will get an instance of *WorkflowExecutionError. This error will contain one of errors above. +*/ + +type ( + // ApplicationErrorOptions represents a combination of error attributes and additional requests. + // All fields are optional, providing flexibility in error customization. + ApplicationErrorOptions struct { + // NonRetryable indicates if the error should not be retried regardless of the retry policy. + NonRetryable bool + // Cause is the original error that caused this error. + Cause error + // Details is a list of arbitrary values that can be used to provide additional context to the error. + Details []interface{} + // NextRetryInterval is a request from server to override retry interval calculated by the + // server according to the RetryPolicy set by the Workflow. + // It is impossible to specify immediate retry as it is indistinguishable from the default value. As a + // workaround you could set NextRetryDelay to some small value. + // NOTE: This option is supported by Temporal Server >= v1.24.2 older version will ignore this value. + NextRetryDelay time.Duration + } + + // ApplicationError returned from activity implementations with message and optional details. + ApplicationError struct { + temporalError + msg string + errType string + nonRetryable bool + cause error + details converter.EncodedValues + nextRetryDelay time.Duration + } + + // TimeoutError returned when activity or child workflow timed out. + TimeoutError struct { + temporalError + msg string + timeoutType enumspb.TimeoutType + lastHeartbeatDetails converter.EncodedValues + cause error + } + + // CanceledError returned when operation was canceled. + CanceledError struct { + temporalError + details converter.EncodedValues + } + + // TerminatedError returned when workflow was terminated. + TerminatedError struct { + temporalError + } + + // PanicError contains information about panicked workflow/activity. + PanicError struct { + temporalError + value interface{} + stackTrace string + } + + // workflowPanicError contains information about panicked workflow. + // Used to distinguish go panic in the workflow code from a PanicError returned from a workflow function. + workflowPanicError struct { + value interface{} + stackTrace string + } + + // ContinueAsNewError contains information about how to continue the workflow as new. + ContinueAsNewError struct { + // params *ExecuteWorkflowParams + WorkflowType *WorkflowType + Input *commonpb.Payloads + Header *commonpb.Header + TaskQueueName string + WorkflowRunTimeout time.Duration + WorkflowTaskTimeout time.Duration + + // Deprecated: WorkflowExecutionTimeout is deprecated and is never set or + // used internally. + WorkflowExecutionTimeout time.Duration + + // VersioningIntent specifies whether the continued workflow should run on a worker with a + // compatible build ID or not. See VersioningIntent. + VersioningIntent VersioningIntent + + // This is by default nil but may be overridden using NewContinueAsNewErrorWithOptions. + // It specifies the retry policy which gets carried over to the next run. + // If not set, the current workflow's retry policy will be carried over automatically. + // + // NOTES: + // 1. This is always nil when returned from a client as a workflow response. + // 2. Unlike other options that can be overridden using WithWorkflowTaskQueue, WithWorkflowRunTimeout, etc. + // we can't introduce an option, say WithWorkflowRetryPolicy, for backward compatibility. + // See #676 or IntegrationTestSuite::TestContinueAsNewWithWithChildWF for more details. + RetryPolicy *RetryPolicy + } + + // ContinueAsNewErrorOptions specifies optional attributes to be carried over to the next run. + ContinueAsNewErrorOptions struct { + // RetryPolicy specifies the retry policy to be used for the next run. + // If nil, the current workflow's retry policy will be used. + RetryPolicy *RetryPolicy + } + + // UnknownExternalWorkflowExecutionError can be returned when external workflow doesn't exist + UnknownExternalWorkflowExecutionError struct{} + + // ServerError can be returned from server. + ServerError struct { + temporalError + msg string + nonRetryable bool + cause error + } + + // ActivityError is returned from workflow when activity returned an error. + // Unwrap this error to get actual cause. + ActivityError struct { + temporalError + scheduledEventID int64 + startedEventID int64 + identity string + activityType *commonpb.ActivityType + activityID string + retryState enumspb.RetryState + cause error + } + + // ChildWorkflowExecutionError is returned from workflow when child workflow returned an error. + // Unwrap this error to get actual cause. + ChildWorkflowExecutionError struct { + temporalError + namespace string + workflowID string + runID string + workflowType string + initiatedEventID int64 + startedEventID int64 + retryState enumspb.RetryState + cause error + } + + // NexusOperationError is an error returned when a Nexus Operation has failed. + // + // NOTE: Experimental + NexusOperationError struct { + // The raw proto failure object this error was created from. + Failure *failurepb.Failure + // Error message. + Message string + // ID of the NexusOperationScheduled event. + ScheduledEventID int64 + // Endpoint name. + Endpoint string + // Service name. + Service string + // Operation name. + Operation string + // Operation ID - may be empty if the operation completed synchronously. + OperationID string + // Chained cause - typically an ApplicationError or a CanceledError. + Cause error + } + + // ChildWorkflowExecutionAlreadyStartedError is set as the cause of + // ChildWorkflowExecutionError when failure is due the child workflow having + // already started. + ChildWorkflowExecutionAlreadyStartedError struct{} + + // NamespaceNotFoundError is set as the cause when failure is due namespace not found. + NamespaceNotFoundError struct{} + + // WorkflowExecutionError is returned from workflow. + // Unwrap this error to get actual cause. + WorkflowExecutionError struct { + workflowID string + runID string + workflowType string + cause error + } + + // ActivityNotRegisteredError is returned if worker doesn't support activity type. + ActivityNotRegisteredError struct { + activityType string + supportedTypes []string + } + + temporalError struct { + messenger + originalFailure *failurepb.Failure + } + + failureHolder interface { + setFailure(*failurepb.Failure) + failure() *failurepb.Failure + } + + messenger interface { + message() string + } +) + +var ( + // Should be "errorString". + goErrType = reflect.TypeOf(errors.New("")).Elem().Name() + + // ErrNoData is returned when trying to extract strong typed data while there is no data available. + ErrNoData = errors.New("no data available") + + // ErrTooManyArg is returned when trying to extract strong typed data with more arguments than available data. + ErrTooManyArg = errors.New("too many arguments") + + // ErrActivityResultPending is returned from activity's implementation to indicate the activity is not completed when + // activity method returns. Activity needs to be completed by Client.CompleteActivity() separately. For example, if an + // activity require human interaction (like approve an expense report), the activity could return activity.ErrResultPending + // which indicate the activity is not done yet. Then, when the waited human action happened, it needs to trigger something + // that could report the activity completed event to temporal server via Client.CompleteActivity() API. + ErrActivityResultPending = errors.New("not error: do not autocomplete, using Client.CompleteActivity() to complete") + + // ErrScheduleAlreadyRunning is returned if there's already a running (not deleted) Schedule with the same ID + ErrScheduleAlreadyRunning = errors.New("schedule with this ID is already registered") + + // ErrSkipScheduleUpdate is used by a user if they want to skip updating a schedule. + ErrSkipScheduleUpdate = errors.New("skip schedule update") +) + +// NewApplicationError create new instance of *ApplicationError with message, type, and optional details. +func NewApplicationError(msg string, errType string, nonRetryable bool, cause error, details ...interface{}) error { + return NewApplicationErrorWithOptions( + msg, + errType, + ApplicationErrorOptions{NonRetryable: nonRetryable, Cause: cause, Details: details}, + ) +} + +func NewApplicationErrorWithOptions(msg string, errType string, options ApplicationErrorOptions) error { + applicationErr := &ApplicationError{ + msg: msg, + errType: errType, + cause: options.Cause, + nonRetryable: options.NonRetryable, + nextRetryDelay: options.NextRetryDelay, + } + // When return error to user, use EncodedValues as details and data is ready to be decoded by calling Get + details := options.Details + if len(details) == 1 { + if d, ok := details[0].(*EncodedValues); ok { + applicationErr.details = d + return applicationErr + } + } + + // When create error for server, use ErrorDetailsValues as details to hold values and encode later + applicationErr.details = ErrorDetailsValues(details) + return applicationErr +} + +// NewTimeoutError creates TimeoutError instance. +// Use NewHeartbeatTimeoutError to create heartbeat TimeoutError. +func NewTimeoutError(msg string, timeoutType enumspb.TimeoutType, cause error, lastHeartbeatDetails ...interface{}) error { + timeoutErr := &TimeoutError{ + msg: msg, + timeoutType: timeoutType, + cause: cause, + } + + if len(lastHeartbeatDetails) == 1 { + if d, ok := lastHeartbeatDetails[0].(*EncodedValues); ok { + timeoutErr.lastHeartbeatDetails = d + return timeoutErr + } + } + timeoutErr.lastHeartbeatDetails = ErrorDetailsValues(lastHeartbeatDetails) + return timeoutErr +} + +// NewHeartbeatTimeoutError creates TimeoutError instance. +func NewHeartbeatTimeoutError(details ...interface{}) error { + return NewTimeoutError("heartbeat timeout", enumspb.TIMEOUT_TYPE_HEARTBEAT, nil, details...) +} + +// NewCanceledError creates CanceledError instance. +func NewCanceledError(details ...interface{}) error { + if len(details) == 1 { + if d, ok := details[0].(*EncodedValues); ok { + return &CanceledError{details: d} + } + } + return &CanceledError{details: ErrorDetailsValues(details)} +} + +// NewServerError create new instance of *ServerError with message. +func NewServerError(msg string, nonRetryable bool, cause error) error { + return &ServerError{msg: msg, nonRetryable: nonRetryable, cause: cause} +} + +// NewActivityError creates ActivityError instance. +func NewActivityError( + scheduledEventID int64, + startedEventID int64, + identity string, + activityType *commonpb.ActivityType, + activityID string, + retryState enumspb.RetryState, + cause error, +) *ActivityError { + return &ActivityError{ + scheduledEventID: scheduledEventID, + startedEventID: startedEventID, + identity: identity, + activityType: activityType, + activityID: activityID, + retryState: retryState, + cause: cause, + } +} + +// NewChildWorkflowExecutionError creates ChildWorkflowExecutionError instance. +func NewChildWorkflowExecutionError( + namespace string, + workflowID string, + runID string, + workflowType string, + initiatedEventID int64, + startedEventID int64, + retryState enumspb.RetryState, + cause error, +) *ChildWorkflowExecutionError { + return &ChildWorkflowExecutionError{ + namespace: namespace, + workflowID: workflowID, + runID: runID, + workflowType: workflowType, + initiatedEventID: initiatedEventID, + startedEventID: startedEventID, + retryState: retryState, + cause: cause, + } +} + +// NewWorkflowExecutionError creates WorkflowExecutionError instance. +func NewWorkflowExecutionError( + workflowID string, + runID string, + workflowType string, + cause error, +) *WorkflowExecutionError { + return &WorkflowExecutionError{ + workflowID: workflowID, + runID: runID, + workflowType: workflowType, + cause: cause, + } +} + +func (e *temporalError) setFailure(f *failurepb.Failure) { + e.originalFailure = f +} + +func (e *temporalError) failure() *failurepb.Failure { + return e.originalFailure +} + +// IsCanceledError returns whether error in CanceledError. +func IsCanceledError(err error) bool { + var canceledErr *CanceledError + return errors.As(err, &canceledErr) +} + +// NewContinueAsNewError creates ContinueAsNewError instance +// If the workflow main function returns this error then the current execution is ended and +// the new execution with same workflow ID is started automatically with options +// provided to this function. +// +// ctx - use context to override any options for the new workflow like run timeout, task timeout, task queue. +// if not mentioned it would use the defaults that the current workflow is using. +// ctx := WithWorkflowRunTimeout(ctx, 30 * time.Minute) +// ctx := WithWorkflowTaskTimeout(ctx, 5 * time.Second) +// ctx := WithWorkflowTaskQueue(ctx, "example-group") +// wfn - workflow function. for new execution it can be different from the currently running. +// args - arguments for the new workflow. +func NewContinueAsNewError(ctx Context, wfn interface{}, args ...interface{}) error { + i := getWorkflowOutboundInterceptor(ctx) + // Put header on context before executing + ctx = workflowContextWithNewHeader(ctx) + return i.NewContinueAsNewError(ctx, wfn, args...) +} + +// NewContinueAsNewErrorWithOptions creates ContinueAsNewError instance with additional options. +func NewContinueAsNewErrorWithOptions(ctx Context, options ContinueAsNewErrorOptions, wfn interface{}, args ...interface{}) error { + err := NewContinueAsNewError(ctx, wfn, args...) + + var continueAsNewErr *ContinueAsNewError + if errors.As(err, &continueAsNewErr) { + if options.RetryPolicy != nil { + continueAsNewErr.RetryPolicy = options.RetryPolicy + } + } + + return err +} + +func (wc *workflowEnvironmentInterceptor) NewContinueAsNewError( + ctx Context, + wfn interface{}, + args ...interface{}, +) error { + // Validate type and its arguments. + options := getWorkflowEnvOptions(ctx) + if options == nil { + panic("context is missing required options for continue as new") + } + env := getWorkflowEnvironment(ctx) + workflowType, input, err := getValidatedWorkflowFunction(wfn, args, options.DataConverter, env.GetRegistry()) + if err != nil { + panic(err) + } + + header, err := workflowHeaderPropagated(ctx, options.ContextPropagators) + if err != nil { + return err + } + + return &ContinueAsNewError{ + WorkflowType: workflowType, + Input: input, + Header: header, + TaskQueueName: options.TaskQueueName, + WorkflowExecutionTimeout: options.WorkflowExecutionTimeout, + WorkflowRunTimeout: options.WorkflowRunTimeout, + WorkflowTaskTimeout: options.WorkflowTaskTimeout, + VersioningIntent: options.VersioningIntent, + RetryPolicy: nil, // The retry policy can't be propagated like other options due to #676. + } +} + +// NewActivityNotRegisteredError creates a new ActivityNotRegisteredError. +func NewActivityNotRegisteredError(activityType string, supportedTypes []string) error { + return &ActivityNotRegisteredError{activityType: activityType, supportedTypes: supportedTypes} +} + +// Error from error interface. +func (e *ApplicationError) Error() string { + msg := e.message() + if e.errType != "" { + msg = fmt.Sprintf("%s (type: %s, retryable: %v)", msg, e.errType, !e.nonRetryable) + } + if e.cause != nil { + msg = fmt.Sprintf("%s: %v", msg, e.cause) + } + return msg +} + +func (e *ApplicationError) message() string { + return e.msg +} + +// Message contains just the message string without extras added by Error(). +func (e *ApplicationError) Message() string { + return e.msg +} + +// Type returns error type represented as string. +// This type can be passed explicitly to ApplicationError constructor. +// Also any other Go error is converted to ApplicationError and type is set automatically using reflection. +// For example instance of "MyCustomError struct" will be converted to ApplicationError and Type() will return "MyCustomError" string. +func (e *ApplicationError) Type() string { + return e.errType +} + +// HasDetails return if this error has strong typed detail data. +func (e *ApplicationError) HasDetails() bool { + return e.details != nil && e.details.HasValues() +} + +// Details extracts strong typed detail data of this custom error. If there is no details, it will return ErrNoData. +func (e *ApplicationError) Details(d ...interface{}) error { + if !e.HasDetails() { + return ErrNoData + } + return e.details.Get(d...) +} + +// NonRetryable indicated if error is not retryable. +func (e *ApplicationError) NonRetryable() bool { + return e.nonRetryable +} + +func (e *ApplicationError) Unwrap() error { + return e.cause +} + +// NextRetryDelay returns the delay to wait before retrying the activity. +// a zero value means to use the activities retry policy. +func (e *ApplicationError) NextRetryDelay() time.Duration { return e.nextRetryDelay } + +// Error from error interface +func (e *TimeoutError) Error() string { + msg := fmt.Sprintf("%s (type: %s)", e.message(), e.timeoutType) + if e.cause != nil { + msg = fmt.Sprintf("%s: %v", msg, e.cause) + } + return msg +} + +func (e *TimeoutError) message() string { + return e.msg +} + +// Message contains just the message string without extras added by Error(). +func (e *TimeoutError) Message() string { + return e.msg +} + +func (e *TimeoutError) Unwrap() error { + return e.cause +} + +// TimeoutType return timeout type of this error +func (e *TimeoutError) TimeoutType() enumspb.TimeoutType { + return e.timeoutType +} + +// HasLastHeartbeatDetails return if this error has strong typed detail data. +func (e *TimeoutError) HasLastHeartbeatDetails() bool { + return e.lastHeartbeatDetails != nil && e.lastHeartbeatDetails.HasValues() +} + +// LastHeartbeatDetails extracts strong typed detail data of this error. If there is no details, it will return ErrNoData. +func (e *TimeoutError) LastHeartbeatDetails(d ...interface{}) error { + if !e.HasLastHeartbeatDetails() { + return ErrNoData + } + return e.lastHeartbeatDetails.Get(d...) +} + +// Error from error interface +func (e *CanceledError) Error() string { + return e.message() +} + +func (e *CanceledError) message() string { + return "canceled" +} + +// HasDetails return if this error has strong typed detail data. +func (e *CanceledError) HasDetails() bool { + return e.details != nil && e.details.HasValues() +} + +// Details extracts strong typed detail data of this error. +func (e *CanceledError) Details(d ...interface{}) error { + if !e.HasDetails() { + return ErrNoData + } + return e.details.Get(d...) +} + +func newPanicError(value interface{}, stackTrace string) error { + return &PanicError{value: value, stackTrace: stackTrace} +} + +func newWorkflowPanicError(value interface{}, stackTrace string) error { + return &workflowPanicError{value: value, stackTrace: stackTrace} +} + +// Error from error interface +func (e *PanicError) Error() string { + return e.message() +} + +func (e *PanicError) message() string { + return fmt.Sprintf("%v", e.value) +} + +// StackTrace return stack trace of the panic +func (e *PanicError) StackTrace() string { + return e.stackTrace +} + +// Error from error interface +func (e *workflowPanicError) Error() string { + return fmt.Sprintf("%v", e.value) +} + +// StackTrace return stack trace of the panic +func (e *workflowPanicError) StackTrace() string { + return e.stackTrace +} + +// Error from error interface +func (e *ContinueAsNewError) Error() string { + return e.message() +} + +func (e *ContinueAsNewError) message() string { + return "continue as new" +} + +// newTerminatedError creates NewTerminatedError instance +func newTerminatedError() *TerminatedError { + return &TerminatedError{} +} + +// Error from error interface +func (e *TerminatedError) Error() string { + return e.message() +} + +func (e *TerminatedError) message() string { + return "terminated" +} + +// newUnknownExternalWorkflowExecutionError creates UnknownExternalWorkflowExecutionError instance +func newUnknownExternalWorkflowExecutionError() *UnknownExternalWorkflowExecutionError { + return &UnknownExternalWorkflowExecutionError{} +} + +// Error from error interface +func (e *UnknownExternalWorkflowExecutionError) Error() string { + return "unknown external workflow execution" +} + +// Error from error interface +func (e *ServerError) Error() string { + msg := e.message() + if e.cause != nil { + msg = fmt.Sprintf("%s: %v", msg, e.cause) + } + return msg +} + +func (e *ServerError) message() string { + return e.msg +} + +// Message contains just the message string without extras added by Error(). +func (e *ServerError) Message() string { + return e.msg +} + +func (e *ServerError) Unwrap() error { + return e.cause +} + +func (e *ActivityError) Error() string { + msg := fmt.Sprintf("%s (type: %s, scheduledEventID: %d, startedEventID: %d, identity: %s)", e.message(), e.activityType.GetName(), e.scheduledEventID, e.startedEventID, e.identity) + if e.cause != nil { + msg = fmt.Sprintf("%s: %v", msg, e.cause) + } + return msg +} + +func (e *ActivityError) message() string { + return "activity error" +} + +func (e *ActivityError) Unwrap() error { + return e.cause +} + +// ScheduledEventID returns event id of the scheduled workflow task corresponding to the activity. +func (e *ActivityError) ScheduledEventID() int64 { + return e.scheduledEventID +} + +// StartedEventID returns event id of the started workflow task corresponding to the activity. +func (e *ActivityError) StartedEventID() int64 { + return e.startedEventID +} + +// Identity returns identity of the worker that attempted activity execution. +func (e *ActivityError) Identity() string { + return e.identity +} + +// ActivityType returns declared type of the activity. +func (e *ActivityError) ActivityType() *commonpb.ActivityType { + return e.activityType +} + +// ActivityID return assigned identifier for the activity. +func (e *ActivityError) ActivityID() string { + return e.activityID +} + +// RetryState returns details on why activity failed. +func (e *ActivityError) RetryState() enumspb.RetryState { + return e.retryState +} + +// Error from error interface +func (e *ChildWorkflowExecutionError) Error() string { + msg := fmt.Sprintf("%s (type: %s, workflowID: %s, runID: %s, initiatedEventID: %d, startedEventID: %d)", + e.message(), e.workflowType, e.workflowID, e.runID, e.initiatedEventID, e.startedEventID) + if e.cause != nil { + msg = fmt.Sprintf("%s: %v", msg, e.cause) + } + return msg +} + +func (e *ChildWorkflowExecutionError) message() string { + return "child workflow execution error" +} + +func (e *ChildWorkflowExecutionError) Unwrap() error { + return e.cause +} + +// Error implements the error interface. +func (e *NexusOperationError) Error() string { + msg := fmt.Sprintf( + "%s (endpoint: %q, service: %q, operation: %q, operation ID: %q, scheduledEventID: %d)", + e.Message, e.Endpoint, e.Service, e.Operation, e.OperationID, e.ScheduledEventID) + if e.Cause != nil { + msg = fmt.Sprintf("%s: %v", msg, e.Cause) + } + return msg +} + +// setFailure implements the failureHolder interface for consistency with other failure based errors.. +func (e *NexusOperationError) setFailure(f *failurepb.Failure) { + e.Failure = f +} + +// failure implements the failureHolder interface for consistency with other failure based errors. +func (e *NexusOperationError) failure() *failurepb.Failure { + return e.Failure +} + +// Unwrap returns the Cause associated with this error. +func (e *NexusOperationError) Unwrap() error { + return e.Cause +} + +// Error from error interface +func (*NamespaceNotFoundError) Error() string { + return "namespace not found" +} + +// Error from error interface +func (*ChildWorkflowExecutionAlreadyStartedError) Error() string { + return "child workflow execution already started" +} + +// Error from error interface +func (e *WorkflowExecutionError) Error() string { + msg := fmt.Sprintf("workflow execution error (type: %s, workflowID: %s, runID: %s)", + e.workflowType, e.workflowID, e.runID) + if e.cause != nil { + msg = fmt.Sprintf("%s: %v", msg, e.cause) + } + return msg +} + +func (e *WorkflowExecutionError) Unwrap() error { + return e.cause +} + +func (e *ActivityNotRegisteredError) Error() string { + supported := strings.Join(e.supportedTypes, ", ") + return fmt.Sprintf("unable to find activityType=%v. Supported types: [%v]", e.activityType, supported) +} + +func convertErrDetailsToPayloads(details converter.EncodedValues, dc converter.DataConverter) *commonpb.Payloads { + switch d := details.(type) { + case ErrorDetailsValues: + data, err := encodeArgs(dc, d) + if err != nil { + panic(err) + } + return data + case *EncodedValues: + return d.values + default: + panic(fmt.Sprintf("unknown error details type %T", details)) + } +} + +// IsRetryable returns if error retryable or not. +func IsRetryable(err error, nonRetryableTypes []string) bool { + if err == nil { + return false + } + + var terminatedErr *TerminatedError + var canceledErr *CanceledError + var workflowPanicErr *workflowPanicError + if errors.As(err, &terminatedErr) || errors.As(err, &canceledErr) || errors.As(err, &workflowPanicErr) { + return false + } + + var timeoutErr *TimeoutError + if errors.As(err, &timeoutErr) { + return timeoutErr.timeoutType == enumspb.TIMEOUT_TYPE_START_TO_CLOSE || timeoutErr.timeoutType == enumspb.TIMEOUT_TYPE_HEARTBEAT + } + + var applicationErr *ApplicationError + var errType string + if errors.As(err, &applicationErr) { + if applicationErr.nonRetryable { + return false + } + errType = applicationErr.errType + } else { + // If it is generic Go error. + errType = getErrType(err) + } + + for _, nonRetryableType := range nonRetryableTypes { + if nonRetryableType == errType { + return false + } + } + + return true +} + +func getErrType(err error) string { + var t reflect.Type + for t = reflect.TypeOf(err); t.Kind() == reflect.Ptr; t = t.Elem() { + } + + if t.Name() == goErrType { + return "" + } + + return t.Name() +} diff --git a/vendor/go.temporal.io/sdk/internal/failure_converter.go b/vendor/go.temporal.io/sdk/internal/failure_converter.go new file mode 100644 index 00000000000..c306f1cc3f7 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/failure_converter.go @@ -0,0 +1,289 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "errors" + "time" + + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/durationpb" + + commonpb "go.temporal.io/api/common/v1" + failurepb "go.temporal.io/api/failure/v1" + + "go.temporal.io/sdk/converter" +) + +var defaultFailureConverter = NewDefaultFailureConverter(DefaultFailureConverterOptions{}) + +// GetDefaultFailureConverter returns the default failure converter used by Temporal. +func GetDefaultFailureConverter() converter.FailureConverter { + return defaultFailureConverter +} + +// DefaultFailureConverterOptions are optional parameters for DefaultFailureConverter creation. +type DefaultFailureConverterOptions struct { + // Optional: Sets DataConverter to customize serialization/deserialization of fields. + // default: Default data converter + DataConverter converter.DataConverter + + // Optional: Whether to encode error messages and stack traces. + // default: false + EncodeCommonAttributes bool +} + +// DefaultFailureConverter seralizes errors with the option to encode common parameters under Failure.EncodedAttributes +type DefaultFailureConverter struct { + dataConverter converter.DataConverter + encodeCommonAttributes bool +} + +// NewDefaultFailureConverter creates new instance of DefaultFailureConverter. +func NewDefaultFailureConverter(opt DefaultFailureConverterOptions) *DefaultFailureConverter { + if opt.DataConverter == nil { + opt.DataConverter = converter.GetDefaultDataConverter() + } + return &DefaultFailureConverter{ + dataConverter: opt.DataConverter, + encodeCommonAttributes: opt.EncodeCommonAttributes, + } +} + +// ErrorToFailure converts an error to a Failure +func (dfc *DefaultFailureConverter) ErrorToFailure(err error) *failurepb.Failure { + if err == nil { + return nil + } + + if fh, ok := err.(failureHolder); ok { + if fh.failure() != nil { + return fh.failure() + } + } + + failure := &failurepb.Failure{ + Source: "GoSDK", + } + + if m, ok := err.(messenger); ok && m != nil { + failure.Message = m.message() + } else { + failure.Message = err.Error() + } + + switch err := err.(type) { + case *ApplicationError: + var delay *durationpb.Duration + if err.nextRetryDelay != 0 { + delay = durationpb.New(err.nextRetryDelay) + } + failureInfo := &failurepb.ApplicationFailureInfo{ + Type: err.errType, + NonRetryable: err.NonRetryable(), + Details: convertErrDetailsToPayloads(err.details, dfc.dataConverter), + NextRetryDelay: delay, + } + failure.FailureInfo = &failurepb.Failure_ApplicationFailureInfo{ApplicationFailureInfo: failureInfo} + case *CanceledError: + failureInfo := &failurepb.CanceledFailureInfo{ + Details: convertErrDetailsToPayloads(err.details, dfc.dataConverter), + } + failure.FailureInfo = &failurepb.Failure_CanceledFailureInfo{CanceledFailureInfo: failureInfo} + case *PanicError: + failureInfo := &failurepb.ApplicationFailureInfo{ + Type: getErrType(err), + } + failure.FailureInfo = &failurepb.Failure_ApplicationFailureInfo{ApplicationFailureInfo: failureInfo} + failure.StackTrace = err.StackTrace() + case *workflowPanicError: + failureInfo := &failurepb.ApplicationFailureInfo{ + Type: getErrType(&PanicError{}), + NonRetryable: true, + } + failure.FailureInfo = &failurepb.Failure_ApplicationFailureInfo{ApplicationFailureInfo: failureInfo} + failure.StackTrace = err.StackTrace() + case *TimeoutError: + failureInfo := &failurepb.TimeoutFailureInfo{ + TimeoutType: err.timeoutType, + LastHeartbeatDetails: convertErrDetailsToPayloads(err.lastHeartbeatDetails, dfc.dataConverter), + } + failure.FailureInfo = &failurepb.Failure_TimeoutFailureInfo{TimeoutFailureInfo: failureInfo} + case *TerminatedError: + failureInfo := &failurepb.TerminatedFailureInfo{} + failure.FailureInfo = &failurepb.Failure_TerminatedFailureInfo{TerminatedFailureInfo: failureInfo} + case *ServerError: + failureInfo := &failurepb.ServerFailureInfo{ + NonRetryable: err.nonRetryable, + } + failure.FailureInfo = &failurepb.Failure_ServerFailureInfo{ServerFailureInfo: failureInfo} + case *ActivityError: + failureInfo := &failurepb.ActivityFailureInfo{ + ScheduledEventId: err.scheduledEventID, + StartedEventId: err.startedEventID, + Identity: err.identity, + ActivityType: err.activityType, + ActivityId: err.activityID, + RetryState: err.retryState, + } + failure.FailureInfo = &failurepb.Failure_ActivityFailureInfo{ActivityFailureInfo: failureInfo} + case *ChildWorkflowExecutionError: + failureInfo := &failurepb.ChildWorkflowExecutionFailureInfo{ + Namespace: err.namespace, + WorkflowExecution: &commonpb.WorkflowExecution{ + WorkflowId: err.workflowID, + RunId: err.runID, + }, + WorkflowType: &commonpb.WorkflowType{Name: err.workflowType}, + InitiatedEventId: err.initiatedEventID, + StartedEventId: err.startedEventID, + RetryState: err.retryState, + } + failure.FailureInfo = &failurepb.Failure_ChildWorkflowExecutionFailureInfo{ChildWorkflowExecutionFailureInfo: failureInfo} + case *NexusOperationError: + failureInfo := &failurepb.NexusOperationFailureInfo{ + ScheduledEventId: err.ScheduledEventID, + Endpoint: err.Endpoint, + Service: err.Service, + Operation: err.Operation, + OperationId: err.OperationID, + } + failure.FailureInfo = &failurepb.Failure_NexusOperationExecutionFailureInfo{NexusOperationExecutionFailureInfo: failureInfo} + default: // All unknown errors are considered to be retryable ApplicationFailureInfo. + failureInfo := &failurepb.ApplicationFailureInfo{ + Type: getErrType(err), + NonRetryable: false, + } + failure.FailureInfo = &failurepb.Failure_ApplicationFailureInfo{ApplicationFailureInfo: failureInfo} + } + + failure.Cause = dfc.ErrorToFailure(errors.Unwrap(err)) + + if dfc.encodeCommonAttributes { + err := converter.EncodeCommonFailureAttributes(dfc.dataConverter, failure) + if err != nil { + panic(err) + } + } + return failure +} + +// FailureToError converts an Failure to an error +func (dfc *DefaultFailureConverter) FailureToError(failure *failurepb.Failure) error { + if failure == nil { + return nil + } + // Copy the original future to pass to the failureHolder + originalFailure := proto.Clone(failure).(*failurepb.Failure) + converter.DecodeCommonFailureAttributes(dfc.dataConverter, failure) + + message := failure.GetMessage() + stackTrace := failure.GetStackTrace() + var err error + + if failure.GetApplicationFailureInfo() != nil { + applicationFailureInfo := failure.GetApplicationFailureInfo() + details := newEncodedValues(applicationFailureInfo.GetDetails(), dfc.dataConverter) + switch applicationFailureInfo.GetType() { + case getErrType(&PanicError{}): + err = newPanicError(message, stackTrace) + default: + var nextRetryDelay time.Duration + if delay := applicationFailureInfo.GetNextRetryDelay(); delay != nil { + nextRetryDelay = delay.AsDuration() + } + err = NewApplicationErrorWithOptions( + message, + applicationFailureInfo.GetType(), + ApplicationErrorOptions{ + NonRetryable: applicationFailureInfo.GetNonRetryable(), + Cause: dfc.FailureToError(failure.GetCause()), + Details: []interface{}{details}, + NextRetryDelay: nextRetryDelay, + }, + ) + } + } else if failure.GetCanceledFailureInfo() != nil { + details := newEncodedValues(failure.GetCanceledFailureInfo().GetDetails(), dfc.dataConverter) + err = NewCanceledError(details) + } else if failure.GetTimeoutFailureInfo() != nil { + timeoutFailureInfo := failure.GetTimeoutFailureInfo() + lastHeartbeatDetails := newEncodedValues(timeoutFailureInfo.GetLastHeartbeatDetails(), dfc.dataConverter) + err = NewTimeoutError( + message, + timeoutFailureInfo.GetTimeoutType(), + dfc.FailureToError(failure.GetCause()), + lastHeartbeatDetails) + } else if failure.GetTerminatedFailureInfo() != nil { + err = newTerminatedError() + } else if failure.GetServerFailureInfo() != nil { + err = NewServerError(message, failure.GetServerFailureInfo().GetNonRetryable(), dfc.FailureToError(failure.GetCause())) + } else if failure.GetResetWorkflowFailureInfo() != nil { + err = NewApplicationError(message, "", true, dfc.FailureToError(failure.GetCause()), failure.GetResetWorkflowFailureInfo().GetLastHeartbeatDetails()) + } else if failure.GetActivityFailureInfo() != nil { + activityTaskInfoFailure := failure.GetActivityFailureInfo() + err = NewActivityError( + activityTaskInfoFailure.GetScheduledEventId(), + activityTaskInfoFailure.GetStartedEventId(), + activityTaskInfoFailure.GetIdentity(), + activityTaskInfoFailure.GetActivityType(), + activityTaskInfoFailure.GetActivityId(), + activityTaskInfoFailure.GetRetryState(), + dfc.FailureToError(failure.GetCause()), + ) + } else if failure.GetChildWorkflowExecutionFailureInfo() != nil { + childWorkflowExecutionFailureInfo := failure.GetChildWorkflowExecutionFailureInfo() + err = NewChildWorkflowExecutionError( + childWorkflowExecutionFailureInfo.GetNamespace(), + childWorkflowExecutionFailureInfo.GetWorkflowExecution().GetWorkflowId(), + childWorkflowExecutionFailureInfo.GetWorkflowExecution().GetRunId(), + childWorkflowExecutionFailureInfo.GetWorkflowType().GetName(), + childWorkflowExecutionFailureInfo.GetInitiatedEventId(), + childWorkflowExecutionFailureInfo.GetStartedEventId(), + childWorkflowExecutionFailureInfo.GetRetryState(), + dfc.FailureToError(failure.GetCause()), + ) + } else if info := failure.GetNexusOperationExecutionFailureInfo(); info != nil { + err = &NexusOperationError{ + Message: failure.Message, + Cause: dfc.FailureToError(failure.GetCause()), + Failure: originalFailure, + ScheduledEventID: info.GetScheduledEventId(), + Endpoint: info.GetEndpoint(), + Service: info.GetService(), + Operation: info.GetOperation(), + OperationID: info.GetOperationId(), + } + } + + if err == nil { + // All unknown types are considered to be retryable ApplicationError. + err = NewApplicationError(message, "", false, dfc.FailureToError(failure.GetCause())) + } + + if fh, ok := err.(failureHolder); ok { + fh.setFailure(originalFailure) + } + + return err +} diff --git a/vendor/go.temporal.io/sdk/internal/grpc_dialer.go b/vendor/go.temporal.io/sdk/internal/grpc_dialer.go new file mode 100644 index 00000000000..c0d44702df6 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/grpc_dialer.go @@ -0,0 +1,207 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "sync/atomic" + "time" + + grpc_retry "github.com/grpc-ecosystem/go-grpc-middleware/retry" + "go.temporal.io/api/serviceerror" + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/internal/common/retry" + "google.golang.org/grpc" + "google.golang.org/grpc/backoff" + "google.golang.org/grpc/credentials" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/keepalive" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" +) + +type ( + // dialParameters are passed to GRPCDialer and must be used to create gRPC connection. + dialParameters struct { + HostPort string + UserConnectionOptions ConnectionOptions + RequiredInterceptors []grpc.UnaryClientInterceptor + DefaultServiceConfig string + } +) + +const ( + // LocalHostPort is a default host:port for worker and client to connect to. + LocalHostPort = "localhost:7233" + + // defaultServiceConfig is a default gRPC connection service config which enables DNS round-robin between IPs. + defaultServiceConfig = `{"loadBalancingConfig": [{"round_robin":{}}]}` + + // minConnectTimeout is the minimum amount of time we are willing to give a connection to complete. + minConnectTimeout = 20 * time.Second + + // attemptSuffix is a suffix added to the metric name for individual call attempts made to the server, which includes retries. + attemptSuffix = "_attempt" + + // mb is a number of bytes in a megabyte + mb = 1024 * 1024 + + // defaultMaxPayloadSize is a maximum size of the payload that grpc client would allow. + defaultMaxPayloadSize = 128 * mb + + // defaultKeepAliveTime is the keep alive time if one is not specified. + defaultKeepAliveTime = 30 * time.Second + + // defaultKeepAliveTimeout is the keep alive timeout if one is not specified. + defaultKeepAliveTimeout = 15 * time.Second +) + +func dial(params dialParameters) (*grpc.ClientConn, error) { + var securityOptions []grpc.DialOption + if params.UserConnectionOptions.TLS != nil { + securityOptions = []grpc.DialOption{ + grpc.WithTransportCredentials(credentials.NewTLS(params.UserConnectionOptions.TLS)), + } + } else { + securityOptions = []grpc.DialOption{ + grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithAuthority(params.UserConnectionOptions.Authority), + } + } + + maxPayloadSize := defaultMaxPayloadSize + if params.UserConnectionOptions.MaxPayloadSize != 0 { + maxPayloadSize = params.UserConnectionOptions.MaxPayloadSize + } + + // gRPC maintains connection pool inside grpc.ClientConn. + // This connection pool has auto reconnect feature. + // If connection goes down, gRPC will try to reconnect using exponential backoff strategy: + // https://github.com/grpc/grpc/blob/master/doc/connection-backoff.md. + // Default MaxDelay is 120 seconds which is too high. + // Setting it to retryPollOperationMaxInterval here will correlate with poll reconnect interval. + var cp = grpc.ConnectParams{ + Backoff: backoff.DefaultConfig, + MinConnectTimeout: minConnectTimeout, + } + cp.Backoff.BaseDelay = retryPollOperationInitialInterval + cp.Backoff.MaxDelay = retryPollOperationMaxInterval + opts := []grpc.DialOption{ + grpc.WithChainUnaryInterceptor(params.RequiredInterceptors...), + grpc.WithDefaultServiceConfig(params.DefaultServiceConfig), + grpc.WithConnectParams(cp), + } + + opts = append(opts, securityOptions...) + opts = append(opts, grpc.WithDefaultCallOptions(grpc.MaxCallSendMsgSize(maxPayloadSize))) + opts = append(opts, grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(maxPayloadSize))) + + if !params.UserConnectionOptions.DisableKeepAliveCheck { + // gRPC utilizes keep alive mechanism to detect dead connections in case if server didn't close them + // gracefully. Client would ping the server periodically and expect replies withing the specified timeout. + // Learn more by reading https://github.com/grpc/grpc/blob/master/doc/keepalive.md + keepAliveTime := params.UserConnectionOptions.KeepAliveTime + if keepAliveTime == 0 { + keepAliveTime = defaultKeepAliveTime + } + keepAliveTimeout := params.UserConnectionOptions.KeepAliveTimeout + if keepAliveTimeout == 0 { + keepAliveTimeout = defaultKeepAliveTimeout + } + var kap = keepalive.ClientParameters{ + Time: keepAliveTime, + Timeout: keepAliveTimeout, + PermitWithoutStream: !params.UserConnectionOptions.DisableKeepAlivePermitWithoutStream, + } + opts = append(opts, grpc.WithKeepaliveParams(kap)) + } + + // Append any user-supplied options + opts = append(opts, params.UserConnectionOptions.DialOptions...) + + return grpc.NewClient(params.HostPort, opts...) +} + +func requiredInterceptors( + clientOptions *ClientOptions, + excludeInternalFromRetry *atomic.Bool, +) []grpc.UnaryClientInterceptor { + interceptors := []grpc.UnaryClientInterceptor{ + errorInterceptor, + // Report aggregated metrics for the call, this is done outside of the retry loop. + metrics.NewGRPCInterceptor(clientOptions.MetricsHandler, "", clientOptions.DisableErrorCodeMetricTags), + // By default the grpc retry interceptor *is disabled*, preventing accidental use of retries. + // We add call options for retry configuration based on the values present in the context. + retry.NewRetryOptionsInterceptor(excludeInternalFromRetry), + // Performs retries *IF* retry options are set for the call. + grpc_retry.UnaryClientInterceptor(), + // Report metrics for every call made to the server. + metrics.NewGRPCInterceptor(clientOptions.MetricsHandler, attemptSuffix, clientOptions.DisableErrorCodeMetricTags), + } + if clientOptions.HeadersProvider != nil { + interceptors = append(interceptors, headersProviderInterceptor(clientOptions.HeadersProvider)) + } + if clientOptions.TrafficController != nil { + interceptors = append(interceptors, trafficControllerInterceptor(clientOptions.TrafficController)) + } + // Add credentials interceptor. This is intentionally added after headers + // provider to overwrite anything set there. + if clientOptions.Credentials != nil { + if interceptor := clientOptions.Credentials.gRPCInterceptor(); interceptor != nil { + interceptors = append(interceptors, interceptor) + } + } + return interceptors +} + +func trafficControllerInterceptor(controller TrafficController) grpc.UnaryClientInterceptor { + return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + err := controller.CheckCallAllowed(ctx, method, req, reply) + // Break execution chain and return an error without sending actual request to the server. + if err != nil { + return err + } + return invoker(ctx, method, req, reply, cc, opts...) + } +} + +func headersProviderInterceptor(headersProvider HeadersProvider) grpc.UnaryClientInterceptor { + return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + headers, err := headersProvider.GetHeaders(ctx) + if err != nil { + return err + } + for k, v := range headers { + ctx = metadata.AppendToOutgoingContext(ctx, k, v) + } + return invoker(ctx, method, req, reply, cc, opts...) + } +} + +func errorInterceptor(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + err := invoker(ctx, method, req, reply, cc, opts...) + err = serviceerror.FromStatus(status.Convert(err)) + return err +} diff --git a/vendor/go.temporal.io/sdk/internal/headers.go b/vendor/go.temporal.io/sdk/internal/headers.go new file mode 100644 index 00000000000..ec98eb33a24 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/headers.go @@ -0,0 +1,145 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + + "go.temporal.io/sdk/converter" + + commonpb "go.temporal.io/api/common/v1" +) + +// HeaderWriter is an interface to write information to temporal headers +type ( + HeaderWriter interface { + Set(string, *commonpb.Payload) + } + + // HeaderReader is an interface to read information from temporal headers + HeaderReader interface { + Get(string) (*commonpb.Payload, bool) + ForEachKey(handler func(string, *commonpb.Payload) error) error + } + + // ContextPropagator is an interface that determines what information from + // context to pass along + ContextPropagator interface { + // Inject injects information from a Go Context into headers + Inject(context.Context, HeaderWriter) error + + // Extract extracts context information from headers and returns a context + // object + Extract(context.Context, HeaderReader) (context.Context, error) + + // InjectFromWorkflow injects information from workflow context into headers + InjectFromWorkflow(Context, HeaderWriter) error + + // ExtractToWorkflow extracts context information from headers and returns + // a workflow context + ExtractToWorkflow(Context, HeaderReader) (Context, error) + } + + // ContextAware is an optional interface that can be implemented alongside + // DataConverter. This interface allows Temporal to pass Workflow/Activity + // contexts to the DataConverter so that it may tailor it's behaviour. + // + // Note that data converters may be called in non-context-aware situations to + // convert payloads that may not be customized per context. Data converter + // implementers should not expect or require contextual data be present. + ContextAware interface { + WithWorkflowContext(ctx Context) converter.DataConverter + WithContext(ctx context.Context) converter.DataConverter + } + + headerReader struct { + header *commonpb.Header + } +) + +func (hr *headerReader) ForEachKey(handler func(string, *commonpb.Payload) error) error { + if hr.header == nil { + return nil + } + for key, value := range hr.header.Fields { + if err := handler(key, value); err != nil { + return err + } + } + return nil +} + +func (hr *headerReader) Get(key string) (*commonpb.Payload, bool) { + if hr.header == nil { + return nil, false + } + payload, ok := hr.header.Fields[key] + return payload, ok +} + +// NewHeaderReader returns a header reader interface +func NewHeaderReader(header *commonpb.Header) HeaderReader { + return &headerReader{header: header} +} + +type headerWriter struct { + header *commonpb.Header +} + +func (hw *headerWriter) Set(key string, value *commonpb.Payload) { + if hw.header == nil { + return + } + hw.header.Fields[key] = value +} + +// NewHeaderWriter returns a header writer interface +func NewHeaderWriter(header *commonpb.Header) HeaderWriter { + if header != nil && header.Fields == nil { + header.Fields = make(map[string]*commonpb.Payload) + } + return &headerWriter{header: header} +} + +// WithWorkflowContext returns a new DataConverter tailored to the passed Workflow context if +// the DataConverter implements the ContextAware interface. Otherwise the DataConverter is returned +// as-is. +func WithWorkflowContext(ctx Context, dc converter.DataConverter) converter.DataConverter { + if d, ok := dc.(ContextAware); ok { + return d.WithWorkflowContext(ctx) + } + return dc +} + +// WithContext returns a new DataConverter tailored to the passed Workflow/Activity context if +// the DataConverter implements the ContextAware interface. Otherwise the DataConverter is returned +// as-is. This is generally used for Activity context but can be context for a Workflow if we're +// not yet executing the workflow so do not have a workflow.Context. +func WithContext(ctx context.Context, dc converter.DataConverter) converter.DataConverter { + if d, ok := dc.(ContextAware); ok { + return d.WithContext(ctx) + } + return dc +} diff --git a/vendor/go.temporal.io/sdk/internal/interceptor.go b/vendor/go.temporal.io/sdk/internal/interceptor.go new file mode 100644 index 00000000000..41d3fb0c282 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/interceptor.go @@ -0,0 +1,491 @@ +// The MIT License +// +// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "time" + + "github.com/nexus-rpc/sdk-go/nexus" + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + updatepb "go.temporal.io/api/update/v1" + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/log" +) + +// Interceptor is a common interface for all interceptors. See documentation in +// the interceptor package for more details. +type Interceptor interface { + ClientInterceptor + WorkerInterceptor +} + +// WorkerInterceptor is a common interface for all interceptors. See +// documentation in the interceptor package for more details. +type WorkerInterceptor interface { + // InterceptActivity is called before each activity interception needed with + // the next interceptor in the chain. + InterceptActivity(ctx context.Context, next ActivityInboundInterceptor) ActivityInboundInterceptor + + // InterceptWorkflow is called before each workflow interception needed with + // the next interceptor in the chain. + InterceptWorkflow(ctx Context, next WorkflowInboundInterceptor) WorkflowInboundInterceptor + + mustEmbedWorkerInterceptorBase() +} + +// ActivityInboundInterceptor is an interface for all activity calls originating +// from the server. See documentation in the interceptor package for more +// details. +type ActivityInboundInterceptor interface { + // Init is the first call of this interceptor. Implementations can change/wrap + // the outbound interceptor before calling Init on the next interceptor. + Init(outbound ActivityOutboundInterceptor) error + + // ExecuteActivity is called when an activity is to be run on this worker. + // interceptor.Header will return a non-nil map for this context. + ExecuteActivity(ctx context.Context, in *ExecuteActivityInput) (interface{}, error) + + mustEmbedActivityInboundInterceptorBase() +} + +// ExecuteActivityInput is the input to ActivityInboundInterceptor.ExecuteActivity. +type ExecuteActivityInput struct { + Args []interface{} +} + +// ActivityOutboundInterceptor is an interface for all activity calls +// originating from the SDK. See documentation in the interceptor package for +// more details. +type ActivityOutboundInterceptor interface { + // GetInfo intercepts activity.GetInfo. + GetInfo(ctx context.Context) ActivityInfo + + // GetLogger intercepts activity.GetLogger. + GetLogger(ctx context.Context) log.Logger + + // GetMetricsHandler intercepts activity.GetMetricsHandler. + GetMetricsHandler(ctx context.Context) metrics.Handler + + // RecordHeartbeat intercepts activity.RecordHeartbeat. + RecordHeartbeat(ctx context.Context, details ...interface{}) + + // HasHeartbeatDetails intercepts activity.HasHeartbeatDetails. + HasHeartbeatDetails(ctx context.Context) bool + + // GetHeartbeatDetails intercepts activity.GetHeartbeatDetails. + GetHeartbeatDetails(ctx context.Context, d ...interface{}) error + + // GetWorkerStopChannel intercepts activity.GetWorkerStopChannel. + GetWorkerStopChannel(ctx context.Context) <-chan struct{} + + mustEmbedActivityOutboundInterceptorBase() +} + +// WorkflowInboundInterceptor is an interface for all workflow calls originating +// from the server. See documentation in the interceptor package for more +// details. +type WorkflowInboundInterceptor interface { + // Init is the first call of this interceptor. Implementations can change/wrap + // the outbound interceptor before calling Init on the next interceptor. + Init(outbound WorkflowOutboundInterceptor) error + + // ExecuteWorkflow is called when a workflow is to be run on this worker. + // interceptor.WorkflowHeader will return a non-nil map for this context. + ExecuteWorkflow(ctx Context, in *ExecuteWorkflowInput) (interface{}, error) + + // HandleSignal is called when a signal is sent to a workflow on this worker. + // interceptor.WorkflowHeader will return a non-nil map for this context. + HandleSignal(ctx Context, in *HandleSignalInput) error + + // HandleQuery is called when a query is sent to a workflow on this worker. + // interceptor.WorkflowHeader will return a non-nil map for this context. + HandleQuery(ctx Context, in *HandleQueryInput) (interface{}, error) + + // ValidateUpdate is always called prior to executing an update, even if the + // update handler for in.Name was not registered with a validation function + // as part of its optional configuration. The same prohibition against + // mutating workflow state that is demanded of UpdateOptions.Validator + // functions also applies to this function. + // + // NOTE: Experimental + ValidateUpdate(ctx Context, in *UpdateInput) error + + // ExecuteUpdate is called after ValidateUpdate if and only if the latter + // returns nil. interceptor.WorkflowHeader will return a non-nil map for + // this context. ExecuteUpdate is allowed to mutate workflow state and + // perform workflow actions such as scheduling activities, timers, etc. + // + // NOTE: Experimental + ExecuteUpdate(ctx Context, in *UpdateInput) (interface{}, error) + + mustEmbedWorkflowInboundInterceptorBase() +} + +// ExecuteWorkflowInput is the input to +// WorkflowInboundInterceptor.ExecuteWorkflow. +type ExecuteWorkflowInput struct { + Args []interface{} +} + +// HandleSignalInput is the input to WorkflowInboundInterceptor.HandleSignal. +type HandleSignalInput struct { + SignalName string + // Arg is the signal argument. It is presented as a primitive payload since + // the type needed for decode is not available at the time of interception. + Arg *commonpb.Payloads +} + +// UpdateInput carries the name and arguments of a workflow update invocation. +type UpdateInput struct { + Name string + Args []interface{} +} + +// HandleQueryInput is the input to WorkflowInboundInterceptor.HandleQuery. +type HandleQueryInput struct { + QueryType string + Args []interface{} +} + +// ExecuteNexusOperationInput is the input to WorkflowOutboundInterceptor.ExecuteNexusOperation. +// +// NOTE: Experimental +type ExecuteNexusOperationInput struct { + // Client to start the operation with. + Client NexusClient + // Operation name or OperationReference from the Nexus SDK. + Operation any + // Operation input. + Input any + // Options for starting the operation. + Options NexusOperationOptions + // Header to attach to the request. + NexusHeader nexus.Header +} + +// RequestCancelNexusOperationInput is the input to WorkflowOutboundInterceptor.RequestCancelNexusOperation. +// +// NOTE: Experimental +type RequestCancelNexusOperationInput struct { + // Client that was used to start the operation. + Client NexusClient + // Operation name or OperationReference from the Nexus SDK. + Operation any + // Operation ID. May be empty if the operation is synchronous or has not started yet. + ID string + // seq number. For internal use only. + seq int64 +} + +// WorkflowOutboundInterceptor is an interface for all workflow calls +// originating from the SDK. See documentation in the interceptor package for +// more details. +type WorkflowOutboundInterceptor interface { + // Go intercepts workflow.Go. + Go(ctx Context, name string, f func(ctx Context)) Context + + // Await intercepts workflow.Await. + Await(ctx Context, condition func() bool) error + + // AwaitWithTimeout intercepts workflow.AwaitWithTimeout. + AwaitWithTimeout(ctx Context, timeout time.Duration, condition func() bool) (bool, error) + + // AwaitWithOptions intercepts workflow.AwaitWithOptions. + // + // NOTE: Experimental + AwaitWithOptions(ctx Context, options AwaitOptions, condition func() bool) (bool, error) + + // ExecuteActivity intercepts workflow.ExecuteActivity. + // interceptor.WorkflowHeader will return a non-nil map for this context. + ExecuteActivity(ctx Context, activityType string, args ...interface{}) Future + + // ExecuteLocalActivity intercepts workflow.ExecuteLocalActivity. + // interceptor.WorkflowHeader will return a non-nil map for this context. + ExecuteLocalActivity(ctx Context, activityType string, args ...interface{}) Future + + // ExecuteChildWorkflow intercepts workflow.ExecuteChildWorkflow. + // interceptor.WorkflowHeader will return a non-nil map for this context. + ExecuteChildWorkflow(ctx Context, childWorkflowType string, args ...interface{}) ChildWorkflowFuture + + // GetInfo intercepts workflow.GetInfo. + GetInfo(ctx Context) *WorkflowInfo + + // GetTypedSearchAttributes intercepts workflow.GetTypedSearchAttributes. + GetTypedSearchAttributes(ctx Context) SearchAttributes + + // GetCurrentUpdateInfo intercepts workflow.GetCurrentUpdateInfo. + // + // NOTE: Experimental + GetCurrentUpdateInfo(ctx Context) *UpdateInfo + + // GetLogger intercepts workflow.GetLogger. + GetLogger(ctx Context) log.Logger + + // GetMetricsHandler intercepts workflow.GetMetricsHandler. + GetMetricsHandler(ctx Context) metrics.Handler + + // Now intercepts workflow.Now. + Now(ctx Context) time.Time + + // NewTimer intercepts workflow.NewTimer. + NewTimer(ctx Context, d time.Duration) Future + + // NewTimer intercepts workflow.NewTimerWithOptions. + // + // NOTE: Experimental + NewTimerWithOptions(ctx Context, d time.Duration, options TimerOptions) Future + + // Sleep intercepts workflow.Sleep. + Sleep(ctx Context, d time.Duration) (err error) + + // RequestCancelExternalWorkflow intercepts + // workflow.RequestCancelExternalWorkflow. + RequestCancelExternalWorkflow(ctx Context, workflowID, runID string) Future + + // SignalExternalWorkflow intercepts workflow.SignalExternalWorkflow. + // interceptor.WorkflowHeader will return a non-nil map for this context. + SignalExternalWorkflow(ctx Context, workflowID, runID, signalName string, arg interface{}) Future + + // SignalChildWorkflow intercepts + // workflow.ChildWorkflowFuture.SignalChildWorkflow. + // interceptor.WorkflowHeader will return a non-nil map for this context. + SignalChildWorkflow(ctx Context, workflowID, signalName string, arg interface{}) Future + + // UpsertSearchAttributes intercepts workflow.UpsertSearchAttributes. + UpsertSearchAttributes(ctx Context, attributes map[string]interface{}) error + + // UpsertTypedSearchAttributes intercepts workflow.UpsertTypedSearchAttributes. + UpsertTypedSearchAttributes(ctx Context, attributes ...SearchAttributeUpdate) error + + // UpsertMemo intercepts workflow.UpsertMemo. + UpsertMemo(ctx Context, memo map[string]interface{}) error + + // GetSignalChannel intercepts workflow.GetSignalChannel. + GetSignalChannel(ctx Context, signalName string) ReceiveChannel + + // GetSignalChannelWithOptions intercepts workflow.GetSignalChannelWithOptions. + // + // NOTE: Experimental + GetSignalChannelWithOptions(ctx Context, signalName string, options SignalChannelOptions) ReceiveChannel + + // SideEffect intercepts workflow.SideEffect. + SideEffect(ctx Context, f func(ctx Context) interface{}) converter.EncodedValue + + // MutableSideEffect intercepts workflow.MutableSideEffect. + MutableSideEffect( + ctx Context, + id string, + f func(ctx Context) interface{}, + equals func(a, b interface{}) bool, + ) converter.EncodedValue + + // GetVersion intercepts workflow.GetVersion. + GetVersion(ctx Context, changeID string, minSupported, maxSupported Version) Version + + // SetQueryHandler intercepts workflow.SetQueryHandler. + SetQueryHandler(ctx Context, queryType string, handler interface{}) error + + // SetQueryHandlerWithOptions intercepts workflow.SetQueryHandlerWithOptions. + // + // NOTE: Experimental + SetQueryHandlerWithOptions(ctx Context, queryType string, handler interface{}, options QueryHandlerOptions) error + + // SetUpdateHandler intercepts workflow.SetUpdateHandler. + // + // NOTE: Experimental + SetUpdateHandler(ctx Context, updateName string, handler interface{}, opts UpdateHandlerOptions) error + + // IsReplaying intercepts workflow.IsReplaying. + IsReplaying(ctx Context) bool + + // HasLastCompletionResult intercepts workflow.HasLastCompletionResult. + HasLastCompletionResult(ctx Context) bool + + // GetLastCompletionResult intercepts workflow.GetLastCompletionResult. + GetLastCompletionResult(ctx Context, d ...interface{}) error + + // GetLastError intercepts workflow.GetLastError. + GetLastError(ctx Context) error + + // NewContinueAsNewError intercepts workflow.NewContinueAsNewError. + // interceptor.WorkflowHeader will return a non-nil map for this context. + NewContinueAsNewError(ctx Context, wfn interface{}, args ...interface{}) error + + // ExecuteNexusOperation intercepts NexusClient.ExecuteOperation. + // + // NOTE: Experimental + ExecuteNexusOperation(ctx Context, input ExecuteNexusOperationInput) NexusOperationFuture + + // RequestCancelNexusOperation intercepts Nexus Operation cancellation via context. + // + // NOTE: Experimental + RequestCancelNexusOperation(ctx Context, input RequestCancelNexusOperationInput) + + mustEmbedWorkflowOutboundInterceptorBase() +} + +// ClientInterceptor for providing a ClientOutboundInterceptor to intercept +// certain workflow-specific client calls from the SDK. See documentation in the +// interceptor package for more details. +type ClientInterceptor interface { + // This is called on client creation if set via client options + InterceptClient(next ClientOutboundInterceptor) ClientOutboundInterceptor + + mustEmbedClientInterceptorBase() +} + +// ClientOutboundInterceptor is an interface for certain workflow-specific calls +// originating from the SDK. See documentation in the interceptor package for +// more details. +type ClientOutboundInterceptor interface { + // ExecuteWorkflow intercepts client.Client.ExecuteWorkflow. + // interceptor.Header will return a non-nil map for this context. + ExecuteWorkflow(context.Context, *ClientExecuteWorkflowInput) (WorkflowRun, error) + + // CreateSchedule - Intercept a service call to CreateSchedule + CreateSchedule(ctx context.Context, options *ScheduleClientCreateInput) (ScheduleHandle, error) + + // SignalWorkflow intercepts client.Client.SignalWorkflow. + // interceptor.Header will return a non-nil map for this context. + SignalWorkflow(context.Context, *ClientSignalWorkflowInput) error + + // SignalWithStartWorkflow intercepts client.Client.SignalWithStartWorkflow. + // interceptor.Header will return a non-nil map for this context. + SignalWithStartWorkflow(context.Context, *ClientSignalWithStartWorkflowInput) (WorkflowRun, error) + + // CancelWorkflow intercepts client.Client.CancelWorkflow. + CancelWorkflow(context.Context, *ClientCancelWorkflowInput) error + + // TerminateWorkflow intercepts client.Client.TerminateWorkflow. + TerminateWorkflow(context.Context, *ClientTerminateWorkflowInput) error + + // QueryWorkflow intercepts client.Client.QueryWorkflow. + // If the query is rejected, QueryWorkflow will return an QueryRejectedError + // interceptor.Header will return a non-nil map for this context. + QueryWorkflow(context.Context, *ClientQueryWorkflowInput) (converter.EncodedValue, error) + + // UpdateWorkflow intercepts client.Client.UpdateWorkflow + // interceptor.Header will return a non-nil map for this context. + // + // NOTE: Experimental + UpdateWorkflow(context.Context, *ClientUpdateWorkflowInput) (WorkflowUpdateHandle, error) + + // PollWorkflowUpdate requests the outcome of a specific update from the + // server. + // + // NOTE: Experimental + PollWorkflowUpdate(context.Context, *ClientPollWorkflowUpdateInput) (*ClientPollWorkflowUpdateOutput, error) + + mustEmbedClientOutboundInterceptorBase() +} + +// ClientUpdateWorkflowInput is the input to +// ClientOutboundInterceptor.UpdateWorkflow +// +// NOTE: Experimental +type ClientUpdateWorkflowInput struct { + UpdateID string + WorkflowID string + UpdateName string + Args []interface{} + RunID string + FirstExecutionRunID string + WaitForStage WorkflowUpdateStage +} + +// ClientPollWorkflowUpdateInput is the input to +// ClientOutboundInterceptor.PollWorkflowUpdate. +type ClientPollWorkflowUpdateInput struct { + UpdateRef *updatepb.UpdateRef +} + +// ClientPollWorkflowUpdateOutput is the output to +// ClientOutboundInterceptor.PollWorkflowUpdate. +type ClientPollWorkflowUpdateOutput struct { + // Result is the result of the update, if it has completed successfully. + Result converter.EncodedValue + // Error is the result of a failed update. + Error error +} + +// ScheduleClientCreateInput is the input to +// ClientOutboundInterceptor.CreateSchedule. +type ScheduleClientCreateInput struct { + Options *ScheduleOptions +} + +// ClientExecuteWorkflowInput is the input to +// ClientOutboundInterceptor.ExecuteWorkflow. +type ClientExecuteWorkflowInput struct { + Options *StartWorkflowOptions + WorkflowType string + Args []interface{} +} + +// ClientSignalWorkflowInput is the input to +// ClientOutboundInterceptor.SignalWorkflow. +type ClientSignalWorkflowInput struct { + WorkflowID string + RunID string + SignalName string + Arg interface{} +} + +// ClientSignalWithStartWorkflowInput is the input to +// ClientOutboundInterceptor.SignalWithStartWorkflow. +type ClientSignalWithStartWorkflowInput struct { + SignalName string + SignalArg interface{} + Options *StartWorkflowOptions + WorkflowType string + Args []interface{} +} + +// ClientCancelWorkflowInput is the input to +// ClientOutboundInterceptor.CancelWorkflow. +type ClientCancelWorkflowInput struct { + WorkflowID string + RunID string +} + +// ClientTerminateWorkflowInput is the input to +// ClientOutboundInterceptor.TerminateWorkflow. +type ClientTerminateWorkflowInput struct { + WorkflowID string + RunID string + Reason string + Details []interface{} +} + +// ClientQueryWorkflowInput is the input to +// ClientOutboundInterceptor.QueryWorkflow. +type ClientQueryWorkflowInput struct { + WorkflowID string + RunID string + QueryType string + Args []interface{} + QueryRejectCondition enumspb.QueryRejectCondition +} diff --git a/vendor/go.temporal.io/sdk/internal/interceptor_base.go b/vendor/go.temporal.io/sdk/internal/interceptor_base.go new file mode 100644 index 00000000000..7ce4758ed38 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/interceptor_base.go @@ -0,0 +1,533 @@ +// The MIT License +// +// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "time" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/log" +) + +// InterceptorBase is a default implementation of Interceptor meant for +// embedding. See documentation in the interceptor package for more details. +type InterceptorBase struct { + ClientInterceptorBase + WorkerInterceptorBase +} + +// WorkerInterceptorBase is a default implementation of WorkerInterceptor meant +// for embedding. See documentation in the interceptor package for more details. +type WorkerInterceptorBase struct{} + +var _ WorkerInterceptor = &WorkerInterceptorBase{} + +// InterceptActivity implements WorkerInterceptor.InterceptActivity. +func (*WorkerInterceptorBase) InterceptActivity( + ctx context.Context, + next ActivityInboundInterceptor, +) ActivityInboundInterceptor { + return &ActivityInboundInterceptorBase{Next: next} +} + +// InterceptWorkflow implements WorkerInterceptor.InterceptWorkflow. +func (*WorkerInterceptorBase) InterceptWorkflow( + ctx Context, + next WorkflowInboundInterceptor, +) WorkflowInboundInterceptor { + return &WorkflowInboundInterceptorBase{Next: next} +} + +func (*WorkerInterceptorBase) mustEmbedWorkerInterceptorBase() {} + +// ActivityInboundInterceptorBase is a default implementation of +// ActivityInboundInterceptor meant for embedding. See documentation in the +// interceptor package for more details. +type ActivityInboundInterceptorBase struct { + Next ActivityInboundInterceptor +} + +var _ ActivityInboundInterceptor = &ActivityInboundInterceptorBase{} + +// Init implements ActivityInboundInterceptor.Init. +func (a *ActivityInboundInterceptorBase) Init(outbound ActivityOutboundInterceptor) error { + return a.Next.Init(outbound) +} + +// ExecuteActivity implements ActivityInboundInterceptor.ExecuteActivity. +func (a *ActivityInboundInterceptorBase) ExecuteActivity( + ctx context.Context, + in *ExecuteActivityInput, +) (interface{}, error) { + return a.Next.ExecuteActivity(ctx, in) +} + +func (*ActivityInboundInterceptorBase) mustEmbedActivityInboundInterceptorBase() {} + +// ActivityOutboundInterceptorBase is a default implementation of +// ActivityOutboundInterceptor meant for embedding. See documentation in the +// interceptor package for more details. +type ActivityOutboundInterceptorBase struct { + Next ActivityOutboundInterceptor +} + +var _ ActivityOutboundInterceptor = &ActivityOutboundInterceptorBase{} + +// GetInfo implements ActivityOutboundInterceptor.GetInfo. +func (a *ActivityOutboundInterceptorBase) GetInfo(ctx context.Context) ActivityInfo { + return a.Next.GetInfo(ctx) +} + +// GetLogger implements ActivityOutboundInterceptor.GetLogger. +func (a *ActivityOutboundInterceptorBase) GetLogger(ctx context.Context) log.Logger { + return a.Next.GetLogger(ctx) +} + +// GetMetricsHandler implements ActivityOutboundInterceptor.GetMetricsHandler. +func (a *ActivityOutboundInterceptorBase) GetMetricsHandler(ctx context.Context) metrics.Handler { + return a.Next.GetMetricsHandler(ctx) +} + +// RecordHeartbeat implements ActivityOutboundInterceptor.RecordHeartbeat. +func (a *ActivityOutboundInterceptorBase) RecordHeartbeat(ctx context.Context, details ...interface{}) { + a.Next.RecordHeartbeat(ctx, details...) +} + +// HasHeartbeatDetails implements +// ActivityOutboundInterceptor.HasHeartbeatDetails. +func (a *ActivityOutboundInterceptorBase) HasHeartbeatDetails(ctx context.Context) bool { + return a.Next.HasHeartbeatDetails(ctx) +} + +// GetHeartbeatDetails implements +// ActivityOutboundInterceptor.GetHeartbeatDetails. +func (a *ActivityOutboundInterceptorBase) GetHeartbeatDetails(ctx context.Context, d ...interface{}) error { + return a.Next.GetHeartbeatDetails(ctx, d...) +} + +// GetWorkerStopChannel implements +// ActivityOutboundInterceptor.GetWorkerStopChannel. +func (a *ActivityOutboundInterceptorBase) GetWorkerStopChannel(ctx context.Context) <-chan struct{} { + return a.Next.GetWorkerStopChannel(ctx) +} + +func (*ActivityOutboundInterceptorBase) mustEmbedActivityOutboundInterceptorBase() {} + +// WorkflowInboundInterceptorBase is a default implementation of +// WorkflowInboundInterceptor meant for embedding. See documentation in the +// interceptor package for more details. +type WorkflowInboundInterceptorBase struct { + Next WorkflowInboundInterceptor +} + +var _ WorkflowInboundInterceptor = &WorkflowInboundInterceptorBase{} + +// Init implements WorkflowInboundInterceptor.Init. +func (w *WorkflowInboundInterceptorBase) Init(outbound WorkflowOutboundInterceptor) error { + return w.Next.Init(outbound) +} + +// ExecuteWorkflow implements WorkflowInboundInterceptor.ExecuteWorkflow. +func (w *WorkflowInboundInterceptorBase) ExecuteWorkflow(ctx Context, in *ExecuteWorkflowInput) (interface{}, error) { + return w.Next.ExecuteWorkflow(ctx, in) +} + +// HandleSignal implements WorkflowInboundInterceptor.HandleSignal. +func (w *WorkflowInboundInterceptorBase) HandleSignal(ctx Context, in *HandleSignalInput) error { + return w.Next.HandleSignal(ctx, in) +} + +// ExecuteUpdate implements WorkflowInboundInterceptor.ExecuteUpdate. +func (w *WorkflowInboundInterceptorBase) ExecuteUpdate(ctx Context, in *UpdateInput) (interface{}, error) { + return w.Next.ExecuteUpdate(ctx, in) +} + +// ValidateUpdate implements WorkflowInboundInterceptor.ValidateUpdate. +func (w *WorkflowInboundInterceptorBase) ValidateUpdate(ctx Context, in *UpdateInput) error { + return w.Next.ValidateUpdate(ctx, in) +} + +// HandleQuery implements WorkflowInboundInterceptor.HandleQuery. +func (w *WorkflowInboundInterceptorBase) HandleQuery(ctx Context, in *HandleQueryInput) (interface{}, error) { + return w.Next.HandleQuery(ctx, in) +} + +func (*WorkflowInboundInterceptorBase) mustEmbedWorkflowInboundInterceptorBase() {} + +// WorkflowOutboundInterceptorBase is a default implementation of +// WorkflowOutboundInterceptor meant for embedding. See documentation in the +// interceptor package for more details. +type WorkflowOutboundInterceptorBase struct { + Next WorkflowOutboundInterceptor +} + +var _ WorkflowOutboundInterceptor = &WorkflowOutboundInterceptorBase{} + +// Go implements WorkflowOutboundInterceptor.Go. +func (w *WorkflowOutboundInterceptorBase) Go(ctx Context, name string, f func(ctx Context)) Context { + return w.Next.Go(ctx, name, f) +} + +// ExecuteActivity implements WorkflowOutboundInterceptor.ExecuteActivity. +func (w *WorkflowOutboundInterceptorBase) ExecuteActivity(ctx Context, activityType string, args ...interface{}) Future { + return w.Next.ExecuteActivity(ctx, activityType, args...) +} + +// Await implements WorkflowOutboundInterceptor.Await. +func (w *WorkflowOutboundInterceptorBase) Await(ctx Context, condition func() bool) error { + return w.Next.Await(ctx, condition) +} + +// AwaitWithTimeout implements WorkflowOutboundInterceptor.AwaitWithTimeout. +func (w *WorkflowOutboundInterceptorBase) AwaitWithTimeout(ctx Context, timeout time.Duration, condition func() bool) (bool, error) { + return w.Next.AwaitWithTimeout(ctx, timeout, condition) +} + +// AwaitWithOptions implements WorkflowOutboundInterceptor.AwaitWithOptions. +// +// NOTE: Experimental +func (w *WorkflowOutboundInterceptorBase) AwaitWithOptions(ctx Context, options AwaitOptions, condition func() bool) (bool, error) { + return w.Next.AwaitWithOptions(ctx, options, condition) +} + +// ExecuteLocalActivity implements WorkflowOutboundInterceptor.ExecuteLocalActivity. +func (w *WorkflowOutboundInterceptorBase) ExecuteLocalActivity( + ctx Context, + activityType string, + args ...interface{}, +) Future { + return w.Next.ExecuteLocalActivity(ctx, activityType, args...) +} + +// ExecuteChildWorkflow implements WorkflowOutboundInterceptor.ExecuteChildWorkflow. +func (w *WorkflowOutboundInterceptorBase) ExecuteChildWorkflow( + ctx Context, + childWorkflowType string, + args ...interface{}, +) ChildWorkflowFuture { + return w.Next.ExecuteChildWorkflow(ctx, childWorkflowType, args...) +} + +// GetInfo implements WorkflowOutboundInterceptor.GetInfo. +func (w *WorkflowOutboundInterceptorBase) GetInfo(ctx Context) *WorkflowInfo { + return w.Next.GetInfo(ctx) +} + +// GetTypedSearchAttributes implements WorkflowOutboundInterceptor.GetTypedSearchAttributes. +func (w *WorkflowOutboundInterceptorBase) GetTypedSearchAttributes(ctx Context) SearchAttributes { + return w.Next.GetTypedSearchAttributes(ctx) +} + +// GetCurrentUpdateInfo implements WorkflowOutboundInterceptor.GetCurrentUpdateInfo. +func (w *WorkflowOutboundInterceptorBase) GetCurrentUpdateInfo(ctx Context) *UpdateInfo { + return w.Next.GetCurrentUpdateInfo(ctx) +} + +// GetLogger implements WorkflowOutboundInterceptor.GetLogger. +func (w *WorkflowOutboundInterceptorBase) GetLogger(ctx Context) log.Logger { + return w.Next.GetLogger(ctx) +} + +// GetMetricsHandler implements WorkflowOutboundInterceptor.GetMetricsHandler. +func (w *WorkflowOutboundInterceptorBase) GetMetricsHandler(ctx Context) metrics.Handler { + return w.Next.GetMetricsHandler(ctx) +} + +// Now implements WorkflowOutboundInterceptor.Now. +func (w *WorkflowOutboundInterceptorBase) Now(ctx Context) time.Time { + return w.Next.Now(ctx) +} + +// NewTimer implements WorkflowOutboundInterceptor.NewTimer. +func (w *WorkflowOutboundInterceptorBase) NewTimer(ctx Context, d time.Duration) Future { + return w.Next.NewTimer(ctx, d) +} + +// NewTimerWithOptions implements WorkflowOutboundInterceptor.NewTimerWithOptions. +// +// NOTE: Experimental +func (w *WorkflowOutboundInterceptorBase) NewTimerWithOptions( + ctx Context, + d time.Duration, + options TimerOptions, +) Future { + return w.Next.NewTimerWithOptions(ctx, d, options) +} + +// Sleep implements WorkflowOutboundInterceptor.Sleep. +func (w *WorkflowOutboundInterceptorBase) Sleep(ctx Context, d time.Duration) (err error) { + return w.Next.Sleep(ctx, d) +} + +// RequestCancelExternalWorkflow implements +// WorkflowOutboundInterceptor.RequestCancelExternalWorkflow. +func (w *WorkflowOutboundInterceptorBase) RequestCancelExternalWorkflow( + ctx Context, + workflowID string, + runID string, +) Future { + return w.Next.RequestCancelExternalWorkflow(ctx, workflowID, runID) +} + +// SignalExternalWorkflow implements +// WorkflowOutboundInterceptor.SignalExternalWorkflow. +func (w *WorkflowOutboundInterceptorBase) SignalExternalWorkflow( + ctx Context, + workflowID string, + runID string, + signalName string, + arg interface{}, +) Future { + return w.Next.SignalExternalWorkflow(ctx, workflowID, runID, signalName, arg) +} + +// SignalChildWorkflow implements +// WorkflowOutboundInterceptor.SignalChildWorkflow. +func (w *WorkflowOutboundInterceptorBase) SignalChildWorkflow( + ctx Context, + workflowID string, + signalName string, + arg interface{}, +) Future { + return w.Next.SignalChildWorkflow(ctx, workflowID, signalName, arg) +} + +// UpsertSearchAttributes implements +// WorkflowOutboundInterceptor.UpsertSearchAttributes. +func (w *WorkflowOutboundInterceptorBase) UpsertSearchAttributes(ctx Context, attributes map[string]interface{}) error { + return w.Next.UpsertSearchAttributes(ctx, attributes) +} + +// UpsertTypedSearchAttributes implements +// WorkflowOutboundInterceptor.UpsertTypedSearchAttributes. +func (w *WorkflowOutboundInterceptorBase) UpsertTypedSearchAttributes(ctx Context, attributes ...SearchAttributeUpdate) error { + return w.Next.UpsertTypedSearchAttributes(ctx, attributes...) +} + +// UpsertMemo implements +// WorkflowOutboundInterceptor.UpsertMemo. +func (w *WorkflowOutboundInterceptorBase) UpsertMemo(ctx Context, memo map[string]interface{}) error { + return w.Next.UpsertMemo(ctx, memo) +} + +// GetSignalChannel implements WorkflowOutboundInterceptor.GetSignalChannel. +func (w *WorkflowOutboundInterceptorBase) GetSignalChannel(ctx Context, signalName string) ReceiveChannel { + return w.Next.GetSignalChannel(ctx, signalName) +} + +// GetSignalChannelWithOptions implements WorkflowOutboundInterceptor.GetSignalChannelWithOptions. +// +// NOTE: Experimental +func (w *WorkflowOutboundInterceptorBase) GetSignalChannelWithOptions( + ctx Context, + signalName string, + options SignalChannelOptions, +) ReceiveChannel { + return w.Next.GetSignalChannelWithOptions(ctx, signalName, options) +} + +// SideEffect implements WorkflowOutboundInterceptor.SideEffect. +func (w *WorkflowOutboundInterceptorBase) SideEffect( + ctx Context, + f func(ctx Context) interface{}, +) converter.EncodedValue { + return w.Next.SideEffect(ctx, f) +} + +// MutableSideEffect implements WorkflowOutboundInterceptor.MutableSideEffect. +func (w *WorkflowOutboundInterceptorBase) MutableSideEffect( + ctx Context, + id string, + f func(ctx Context) interface{}, + equals func(a, b interface{}) bool, +) converter.EncodedValue { + return w.Next.MutableSideEffect(ctx, id, f, equals) +} + +// GetVersion implements WorkflowOutboundInterceptor.GetVersion. +func (w *WorkflowOutboundInterceptorBase) GetVersion( + ctx Context, + changeID string, + minSupported Version, + maxSupported Version, +) Version { + return w.Next.GetVersion(ctx, changeID, minSupported, maxSupported) +} + +// SetQueryHandler implements WorkflowOutboundInterceptor.SetQueryHandler. +func (w *WorkflowOutboundInterceptorBase) SetQueryHandler(ctx Context, queryType string, handler interface{}) error { + return w.Next.SetQueryHandler(ctx, queryType, handler) +} + +// SetQueryHandlerWithOptions implements WorkflowOutboundInterceptor.SetQueryHandlerWithOptions. +// +// NOTE: Experimental +func (w *WorkflowOutboundInterceptorBase) SetQueryHandlerWithOptions( + ctx Context, + queryType string, + handler interface{}, + options QueryHandlerOptions, +) error { + return w.Next.SetQueryHandlerWithOptions(ctx, queryType, handler, options) +} + +// SetUpdateHandler implements WorkflowOutboundInterceptor.SetUpdateHandler. +func (w *WorkflowOutboundInterceptorBase) SetUpdateHandler(ctx Context, updateName string, handler interface{}, opts UpdateHandlerOptions) error { + return w.Next.SetUpdateHandler(ctx, updateName, handler, opts) +} + +// IsReplaying implements WorkflowOutboundInterceptor.IsReplaying. +func (w *WorkflowOutboundInterceptorBase) IsReplaying(ctx Context) bool { + return w.Next.IsReplaying(ctx) +} + +// HasLastCompletionResult implements +// WorkflowOutboundInterceptor.HasLastCompletionResult. +func (w *WorkflowOutboundInterceptorBase) HasLastCompletionResult(ctx Context) bool { + return w.Next.HasLastCompletionResult(ctx) +} + +// GetLastCompletionResult implements +// WorkflowOutboundInterceptor.GetLastCompletionResult. +func (w *WorkflowOutboundInterceptorBase) GetLastCompletionResult(ctx Context, d ...interface{}) error { + return w.Next.GetLastCompletionResult(ctx, d...) +} + +// GetLastError implements WorkflowOutboundInterceptor.GetLastError. +func (w *WorkflowOutboundInterceptorBase) GetLastError(ctx Context) error { + return w.Next.GetLastError(ctx) +} + +// NewContinueAsNewError implements +// WorkflowOutboundInterceptor.NewContinueAsNewError. +func (w *WorkflowOutboundInterceptorBase) NewContinueAsNewError( + ctx Context, + wfn interface{}, + args ...interface{}, +) error { + return w.Next.NewContinueAsNewError(ctx, wfn, args...) +} + +// ExecuteNexusOperation implements +// WorkflowOutboundInterceptor.ExecuteNexusOperation. +func (w *WorkflowOutboundInterceptorBase) ExecuteNexusOperation( + ctx Context, + input ExecuteNexusOperationInput, +) NexusOperationFuture { + return w.Next.ExecuteNexusOperation(ctx, input) +} + +// RequestCancelNexusOperation implements +// WorkflowOutboundInterceptor.RequestCancelNexusOperation. +func (w *WorkflowOutboundInterceptorBase) RequestCancelNexusOperation(ctx Context, input RequestCancelNexusOperationInput) { + w.Next.RequestCancelNexusOperation(ctx, input) +} + +func (*WorkflowOutboundInterceptorBase) mustEmbedWorkflowOutboundInterceptorBase() {} + +// ClientInterceptorBase is a default implementation of ClientInterceptor meant +// for embedding. See documentation in the interceptor package for more details. +type ClientInterceptorBase struct{} + +var _ ClientInterceptor = &ClientInterceptorBase{} + +// InterceptClient implements ClientInterceptor.InterceptClient. +func (*ClientInterceptorBase) InterceptClient( + next ClientOutboundInterceptor, +) ClientOutboundInterceptor { + return &ClientOutboundInterceptorBase{Next: next} +} + +func (*ClientInterceptorBase) mustEmbedClientInterceptorBase() {} + +// ClientOutboundInterceptorBase is a default implementation of +// ClientOutboundInterceptor meant for embedding. See documentation in the +// interceptor package for more details. +type ClientOutboundInterceptorBase struct { + Next ClientOutboundInterceptor +} + +var _ ClientOutboundInterceptor = &ClientOutboundInterceptorBase{} + +func (c *ClientOutboundInterceptorBase) UpdateWorkflow( + ctx context.Context, + in *ClientUpdateWorkflowInput, +) (WorkflowUpdateHandle, error) { + return c.Next.UpdateWorkflow(ctx, in) +} + +func (c *ClientOutboundInterceptorBase) PollWorkflowUpdate( + ctx context.Context, + in *ClientPollWorkflowUpdateInput, +) (*ClientPollWorkflowUpdateOutput, error) { + return c.Next.PollWorkflowUpdate(ctx, in) +} + +// ExecuteWorkflow implements ClientOutboundInterceptor.ExecuteWorkflow. +func (c *ClientOutboundInterceptorBase) ExecuteWorkflow( + ctx context.Context, + in *ClientExecuteWorkflowInput, +) (WorkflowRun, error) { + return c.Next.ExecuteWorkflow(ctx, in) +} + +// SignalWorkflow implements ClientOutboundInterceptor.SignalWorkflow. +func (c *ClientOutboundInterceptorBase) SignalWorkflow(ctx context.Context, in *ClientSignalWorkflowInput) error { + return c.Next.SignalWorkflow(ctx, in) +} + +// SignalWithStartWorkflow implements +// ClientOutboundInterceptor.SignalWithStartWorkflow. +func (c *ClientOutboundInterceptorBase) SignalWithStartWorkflow( + ctx context.Context, + in *ClientSignalWithStartWorkflowInput, +) (WorkflowRun, error) { + return c.Next.SignalWithStartWorkflow(ctx, in) +} + +// CancelWorkflow implements ClientOutboundInterceptor.CancelWorkflow. +func (c *ClientOutboundInterceptorBase) CancelWorkflow(ctx context.Context, in *ClientCancelWorkflowInput) error { + return c.Next.CancelWorkflow(ctx, in) +} + +// TerminateWorkflow implements ClientOutboundInterceptor.TerminateWorkflow. +func (c *ClientOutboundInterceptorBase) TerminateWorkflow(ctx context.Context, in *ClientTerminateWorkflowInput) error { + return c.Next.TerminateWorkflow(ctx, in) +} + +// QueryWorkflow implements ClientOutboundInterceptor.QueryWorkflow. +func (c *ClientOutboundInterceptorBase) QueryWorkflow( + ctx context.Context, + in *ClientQueryWorkflowInput, +) (converter.EncodedValue, error) { + return c.Next.QueryWorkflow(ctx, in) +} + +// ExecuteWorkflow implements ClientOutboundInterceptor.CreateSchedule. +func (c *ClientOutboundInterceptorBase) CreateSchedule(ctx context.Context, in *ScheduleClientCreateInput) (ScheduleHandle, error) { + return c.Next.CreateSchedule(ctx, in) +} + +func (*ClientOutboundInterceptorBase) mustEmbedClientOutboundInterceptorBase() {} diff --git a/vendor/go.temporal.io/sdk/internal/interceptor_header.go b/vendor/go.temporal.io/sdk/internal/interceptor_header.go new file mode 100644 index 00000000000..b0a379816a9 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/interceptor_header.go @@ -0,0 +1,134 @@ +// The MIT License +// +// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "fmt" + + commonpb "go.temporal.io/api/common/v1" +) + +type headerKey struct{} + +// Header provides Temporal header information from the context for reading or +// writing during specific interceptor calls. See documentation in the +// interceptor package for more details. +func Header(ctx context.Context) map[string]*commonpb.Payload { + m, _ := ctx.Value(headerKey{}).(map[string]*commonpb.Payload) + return m +} + +func contextWithNewHeader(ctx context.Context) context.Context { + return context.WithValue(ctx, headerKey{}, map[string]*commonpb.Payload{}) +} + +func contextWithoutHeader(ctx context.Context) context.Context { + return context.WithValue(ctx, headerKey{}, nil) +} + +func contextWithHeaderPropagated( + ctx context.Context, + header *commonpb.Header, + ctxProps []ContextPropagator, +) (context.Context, error) { + if header == nil { + header = &commonpb.Header{} + } + if header.Fields == nil { + header.Fields = map[string]*commonpb.Payload{} + } + reader := NewHeaderReader(header) + for _, ctxProp := range ctxProps { + var err error + if ctx, err = ctxProp.Extract(ctx, reader); err != nil { + return nil, fmt.Errorf("failed propagating header: %w", err) + } + } + return context.WithValue(ctx, headerKey{}, header.Fields), nil +} + +func headerPropagated(ctx context.Context, ctxProps []ContextPropagator) (*commonpb.Header, error) { + header := &commonpb.Header{Fields: Header(ctx)} + if header.Fields == nil { + return nil, fmt.Errorf("context missing header") + } + writer := NewHeaderWriter(header) + for _, ctxProp := range ctxProps { + if err := ctxProp.Inject(ctx, writer); err != nil { + return nil, fmt.Errorf("failed propagating header: %w", err) + } + } + return header, nil +} + +// WorkflowHeader provides Temporal header information from the workflow context +// for reading or writing during specific interceptor calls. See documentation +// in the interceptor package for more details. +func WorkflowHeader(ctx Context) map[string]*commonpb.Payload { + m, _ := ctx.Value(headerKey{}).(map[string]*commonpb.Payload) + return m +} + +func workflowContextWithNewHeader(ctx Context) Context { + return WithValue(ctx, headerKey{}, map[string]*commonpb.Payload{}) +} + +func workflowContextWithoutHeader(ctx Context) Context { + return WithValue(ctx, headerKey{}, nil) +} + +func workflowContextWithHeaderPropagated( + ctx Context, + header *commonpb.Header, + ctxProps []ContextPropagator, +) (Context, error) { + if header == nil { + header = &commonpb.Header{} + } + if header.Fields == nil { + header.Fields = map[string]*commonpb.Payload{} + } + reader := NewHeaderReader(header) + for _, ctxProp := range ctxProps { + var err error + if ctx, err = ctxProp.ExtractToWorkflow(ctx, reader); err != nil { + return nil, fmt.Errorf("failed propagating header: %w", err) + } + } + return WithValue(ctx, headerKey{}, header.Fields), nil +} + +func workflowHeaderPropagated(ctx Context, ctxProps []ContextPropagator) (*commonpb.Header, error) { + header := &commonpb.Header{Fields: WorkflowHeader(ctx)} + if header.Fields == nil { + return nil, fmt.Errorf("context missing workflow header") + } + writer := NewHeaderWriter(header) + for _, ctxProp := range ctxProps { + if err := ctxProp.InjectFromWorkflow(ctx, writer); err != nil { + return nil, fmt.Errorf("failed propagating header: %w", err) + } + } + return header, nil +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_activity.go b/vendor/go.temporal.io/sdk/internal/internal_activity.go new file mode 100644 index 00000000000..84c525530a1 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_activity.go @@ -0,0 +1,434 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +// All code in this file is private to the package. + +import ( + "context" + "errors" + "fmt" + "reflect" + "time" + + "google.golang.org/protobuf/proto" + + commonpb "go.temporal.io/api/common/v1" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/log" +) + +type ( + // activity is an interface of an activity implementation. + activity interface { + Execute(ctx context.Context, input *commonpb.Payloads) (*commonpb.Payloads, error) + ActivityType() ActivityType + GetFunction() interface{} + } + + // ActivityID uniquely identifies an activity execution + ActivityID struct { + id string + } + + // LocalActivityID uniquely identifies a local activity execution + LocalActivityID struct { + id string + } + + // ExecuteActivityOptions option for executing an activity + ExecuteActivityOptions struct { + ActivityID string // Users can choose IDs but our framework makes it optional to decrease the crust. + TaskQueueName string + ScheduleToCloseTimeout time.Duration + ScheduleToStartTimeout time.Duration + StartToCloseTimeout time.Duration + HeartbeatTimeout time.Duration + WaitForCancellation bool + OriginalTaskQueueName string + RetryPolicy *commonpb.RetryPolicy + DisableEagerExecution bool + VersioningIntent VersioningIntent + } + + // ExecuteLocalActivityOptions options for executing a local activity + ExecuteLocalActivityOptions struct { + ScheduleToCloseTimeout time.Duration + StartToCloseTimeout time.Duration + RetryPolicy *RetryPolicy + } + + // ExecuteActivityParams parameters for executing an activity + ExecuteActivityParams struct { + ExecuteActivityOptions + ActivityType ActivityType + Input *commonpb.Payloads + DataConverter converter.DataConverter + Header *commonpb.Header + } + + // ExecuteLocalActivityParams parameters for executing a local activity + ExecuteLocalActivityParams struct { + ExecuteLocalActivityOptions + ActivityFn interface{} // local activity function pointer + ActivityType string // local activity type + InputArgs []interface{} + WorkflowInfo *WorkflowInfo + DataConverter converter.DataConverter + Attempt int32 + ScheduledTime time.Time + Header *commonpb.Header + } + + // AsyncActivityClient for requesting activity execution + AsyncActivityClient interface { + // The ExecuteActivity schedules an activity with a callback handler. + // If the activity failed to complete the callback error would indicate the failure + // and it can be one of ActivityTaskFailedError, ActivityTaskTimeoutError, ActivityTaskCanceledError + ExecuteActivity(parameters ExecuteActivityParams, callback ResultHandler) ActivityID + + // This only initiates cancel request for activity. if the activity is configured to not WaitForCancellation then + // it would invoke the callback handler immediately with error code ActivityTaskCanceledError. + // If the activity is not running(either scheduled or started) then it is a no-operation. + RequestCancelActivity(activityID ActivityID) + } + + // LocalActivityClient for requesting local activity execution + LocalActivityClient interface { + ExecuteLocalActivity(params ExecuteLocalActivityParams, callback LocalActivityResultHandler) LocalActivityID + + RequestCancelLocalActivity(activityID LocalActivityID) + } + + activityEnvironment struct { + taskToken []byte + workflowExecution WorkflowExecution + activityID string + activityType ActivityType + serviceInvoker ServiceInvoker + logger log.Logger + metricsHandler metrics.Handler + isLocalActivity bool + heartbeatTimeout time.Duration + deadline time.Time + scheduledTime time.Time + startedTime time.Time + taskQueue string + dataConverter converter.DataConverter + attempt int32 // starts from 1. + heartbeatDetails *commonpb.Payloads + workflowType *WorkflowType + workflowNamespace string + workerStopChannel <-chan struct{} + contextPropagators []ContextPropagator + } + + // context.WithValue need this type instead of basic type string to avoid lint error + contextKey string +) + +const ( + activityEnvContextKey contextKey = "activityEnv" + activityOptionsContextKey contextKey = "activityOptions" + localActivityOptionsContextKey contextKey = "localActivityOptions" + activityInterceptorContextKey contextKey = "activityInterceptor" + activityEnvInterceptorContextKey contextKey = "activityEnvInterceptor" +) + +func (i ActivityID) String() string { + return i.id +} + +// ParseActivityID returns ActivityID constructed from its string representation. +// The string representation should be obtained through ActivityID.String() +func ParseActivityID(id string) (ActivityID, error) { + return ActivityID{id: id}, nil +} + +func (i LocalActivityID) String() string { + return i.id +} + +// ParseLocalActivityID returns LocalActivityID constructed from its string representation. +// The string representation should be obtained through LocalActivityID.String() +func ParseLocalActivityID(v string) (LocalActivityID, error) { + return LocalActivityID{id: v}, nil +} + +func getActivityEnv(ctx context.Context) *activityEnvironment { + env := ctx.Value(activityEnvContextKey) + if env == nil { + panic("getActivityEnv: Not an activity context") + } + return env.(*activityEnvironment) +} + +func getActivityOptions(ctx Context) *ExecuteActivityOptions { + eap := ctx.Value(activityOptionsContextKey) + if eap == nil { + return nil + } + return eap.(*ExecuteActivityOptions) +} + +func getLocalActivityOptions(ctx Context) *ExecuteLocalActivityOptions { + opts := ctx.Value(localActivityOptionsContextKey) + if opts == nil { + return nil + } + return opts.(*ExecuteLocalActivityOptions) +} + +func getValidatedLocalActivityOptions(ctx Context) (*ExecuteLocalActivityOptions, error) { + p := getLocalActivityOptions(ctx) + if p == nil { + return nil, errLocalActivityParamsBadRequest + } + if p.ScheduleToCloseTimeout < 0 { + return nil, errors.New("negative ScheduleToCloseTimeout") + } + if p.StartToCloseTimeout < 0 { + return nil, errors.New("negative StartToCloseTimeout") + } + if p.ScheduleToCloseTimeout == 0 && p.StartToCloseTimeout == 0 { + return nil, errors.New("at least one of ScheduleToCloseTimeout and StartToCloseTimeout is required") + } + if p.ScheduleToCloseTimeout == 0 { + p.ScheduleToCloseTimeout = p.StartToCloseTimeout + } + if p.StartToCloseTimeout == 0 { + p.StartToCloseTimeout = p.ScheduleToCloseTimeout + } + return p, nil +} + +func validateFunctionArgs(workflowFunc interface{}, args []interface{}, isWorkflow bool) error { + fType := reflect.TypeOf(workflowFunc) + switch getKind(fType) { + case reflect.String: + // We can't validate function passed as string. + return nil + case reflect.Func: + default: + return fmt.Errorf( + "invalid type 'workflowFunc' parameter provided, it can be either worker function or function name: %v", + workflowFunc) + } + + fnName, _ := getFunctionName(workflowFunc) + fnArgIndex := 0 + // Skip Context function argument. + if fType.NumIn() > 0 { + if isWorkflow && isWorkflowContext(fType.In(0)) { + fnArgIndex++ + } + if !isWorkflow && isActivityContext(fType.In(0)) { + fnArgIndex++ + } + } + + // Validate provided args match with function order match. + if fType.NumIn()-fnArgIndex != len(args) { + return fmt.Errorf( + "expected %d args for function: %v but found %v", + fType.NumIn()-fnArgIndex, fnName, len(args)) + } + + for i := 0; fnArgIndex < fType.NumIn(); fnArgIndex, i = fnArgIndex+1, i+1 { + fnArgType := fType.In(fnArgIndex) + argType := reflect.TypeOf(args[i]) + if argType != nil && !argType.AssignableTo(fnArgType) { + return fmt.Errorf( + "cannot assign function argument: %d from type: %s to type: %s", + fnArgIndex+1, argType, fnArgType, + ) + } + } + + return nil +} + +func getValidatedActivityFunction(f interface{}, args []interface{}, registry *registry) (*ActivityType, error) { + fnName := "" + fType := reflect.TypeOf(f) + switch getKind(fType) { + case reflect.String: + fnName = reflect.ValueOf(f).String() + case reflect.Func: + if err := validateFunctionArgs(f, args, false); err != nil { + return nil, err + } + fnName, _ = getFunctionName(f) + if alias, ok := registry.getActivityAlias(fnName); ok { + fnName = alias + } + + default: + return nil, fmt.Errorf( + "invalid type 'f' parameter provided, it can be either activity function or name of the activity: %v", f) + } + + return &ActivityType{Name: fnName}, nil +} + +func getKind(fType reflect.Type) reflect.Kind { + if fType == nil { + return reflect.Invalid + } + return fType.Kind() +} + +func isActivityContext(inType reflect.Type) bool { + contextElem := reflect.TypeOf((*context.Context)(nil)).Elem() + return inType != nil && inType.Implements(contextElem) +} + +func setActivityParametersIfNotExist(ctx Context) Context { + params := getActivityOptions(ctx) + var newParams ExecuteActivityOptions + if params != nil { + newParams = *params + if params.RetryPolicy != nil { + newParams.RetryPolicy = proto.Clone(params.RetryPolicy).(*commonpb.RetryPolicy) + } + } + return WithValue(ctx, activityOptionsContextKey, &newParams) +} + +func setLocalActivityParametersIfNotExist(ctx Context) Context { + params := getLocalActivityOptions(ctx) + var newParams ExecuteLocalActivityOptions + if params != nil { + newParams = *params + } + return WithValue(ctx, localActivityOptionsContextKey, &newParams) +} + +type activityEnvironmentInterceptor struct { + env *activityEnvironment + inboundInterceptor ActivityInboundInterceptor + outboundInterceptor ActivityOutboundInterceptor + fn interface{} +} + +func getActivityEnvironmentInterceptor(ctx context.Context) *activityEnvironmentInterceptor { + a := ctx.Value(activityEnvInterceptorContextKey) + if a == nil { + panic("getActivityEnvironmentInterceptor: Not an activity context") + } + return a.(*activityEnvironmentInterceptor) +} + +func getActivityOutboundInterceptor(ctx context.Context) ActivityOutboundInterceptor { + a := ctx.Value(activityInterceptorContextKey) + if a == nil { + panic("getActivityOutboundInterceptor: Not an activity context") + } + return a.(ActivityOutboundInterceptor) +} + +func (a *activityEnvironmentInterceptor) Init(outbound ActivityOutboundInterceptor) error { + a.outboundInterceptor = outbound + return nil +} + +func (a *activityEnvironmentInterceptor) ExecuteActivity( + ctx context.Context, + in *ExecuteActivityInput, +) (interface{}, error) { + // Remove header from context + ctx = contextWithoutHeader(ctx) + + return executeFunctionWithContext(ctx, a.fn, in.Args) +} + +func (a *activityEnvironmentInterceptor) GetInfo(ctx context.Context) ActivityInfo { + return ActivityInfo{ + ActivityID: a.env.activityID, + ActivityType: a.env.activityType, + TaskToken: a.env.taskToken, + WorkflowExecution: a.env.workflowExecution, + HeartbeatTimeout: a.env.heartbeatTimeout, + Deadline: a.env.deadline, + ScheduledTime: a.env.scheduledTime, + StartedTime: a.env.startedTime, + TaskQueue: a.env.taskQueue, + Attempt: a.env.attempt, + WorkflowType: a.env.workflowType, + WorkflowNamespace: a.env.workflowNamespace, + IsLocalActivity: a.env.isLocalActivity, + } +} + +func (a *activityEnvironmentInterceptor) GetLogger(ctx context.Context) log.Logger { + return a.env.logger +} + +func (a *activityEnvironmentInterceptor) GetMetricsHandler(ctx context.Context) metrics.Handler { + return a.env.metricsHandler +} + +func (a *activityEnvironmentInterceptor) RecordHeartbeat(ctx context.Context, details ...interface{}) { + if a.env.isLocalActivity { + // no-op for local activity + return + } + var data *commonpb.Payloads + var err error + // We would like to be able to pass in "nil" as part of details(that is no progress to report to) + if len(details) > 1 || (len(details) == 1 && details[0] != nil) { + data, err = encodeArgs(getDataConverterFromActivityCtx(ctx), details) + if err != nil { + panic(err) + } + } + + // Heartbeat error is logged inside ServiceInvoker.internalHeartBeat + _ = a.env.serviceInvoker.Heartbeat(ctx, data, false) +} + +func (a *activityEnvironmentInterceptor) HasHeartbeatDetails(ctx context.Context) bool { + return a.env.heartbeatDetails != nil +} + +func (a *activityEnvironmentInterceptor) GetHeartbeatDetails(ctx context.Context, d ...interface{}) error { + if a.env.heartbeatDetails == nil { + return ErrNoData + } + encoded := newEncodedValues(a.env.heartbeatDetails, a.env.dataConverter) + return encoded.Get(d...) +} + +func (a *activityEnvironmentInterceptor) GetWorkerStopChannel(ctx context.Context) <-chan struct{} { + return a.env.workerStopChannel +} + +// Needed so this can properly be considered an inbound interceptor +func (a *activityEnvironmentInterceptor) mustEmbedActivityInboundInterceptorBase() {} + +// Needed so this can properly be considered an outbound interceptor +func (a *activityEnvironmentInterceptor) mustEmbedActivityOutboundInterceptorBase() {} diff --git a/vendor/go.temporal.io/sdk/internal/internal_command_state_machine.go b/vendor/go.temporal.io/sdk/internal/internal_command_state_machine.go new file mode 100644 index 00000000000..7515d10b9b4 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_command_state_machine.go @@ -0,0 +1,1670 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "container/list" + "fmt" + "strconv" + + commandpb "go.temporal.io/api/command/v1" + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + failurepb "go.temporal.io/api/failure/v1" + historypb "go.temporal.io/api/history/v1" + "go.temporal.io/api/sdk/v1" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/util" +) + +type ( + commandState int32 + commandType int32 + + commandID struct { + commandType commandType + id string + } + + commandStateMachine interface { + getState() commandState + getID() commandID + isDone() bool + getCommand() *commandpb.Command // return nil if there is no command in current state + cancel() + + handleStartedEvent() + handleCancelInitiatedEvent() + handleCanceledEvent() + handleCancelFailedEvent() + handleCompletionEvent() + handleInitiationFailedEvent() + handleInitiatedEvent() + + handleCommandSent() + setData(data interface{}) + getData() interface{} + } + + commandStateMachineBase struct { + id commandID + state commandState + history []string + data interface{} + helper *commandsHelper + } + + activityCommandStateMachine struct { + *commandStateMachineBase + scheduleID int64 + attributes *commandpb.ScheduleActivityTaskCommandAttributes + } + + cancelActivityStateMachine struct { + *commandStateMachineBase + attributes *commandpb.RequestCancelActivityTaskCommandAttributes + } + + timerCommandStateMachine struct { + *commandStateMachineBase + attributes *commandpb.StartTimerCommandAttributes + startMetadata *sdk.UserMetadata + } + + cancelTimerCommandStateMachine struct { + *commandStateMachineBase + attributes *commandpb.CancelTimerCommandAttributes + } + + childWorkflowCommandStateMachine struct { + *commandStateMachineBase + attributes *commandpb.StartChildWorkflowExecutionCommandAttributes + startMetadata *sdk.UserMetadata + } + + naiveCommandStateMachine struct { + *commandStateMachineBase + command *commandpb.Command + } + + // only possible state transition is: CREATED->SENT->INITIATED->COMPLETED + cancelExternalWorkflowCommandStateMachine struct { + *naiveCommandStateMachine + } + + signalExternalWorkflowCommandStateMachine struct { + *naiveCommandStateMachine + } + + // only possible state transition is: CREATED->SENT->COMPLETED + markerCommandStateMachine struct { + *naiveCommandStateMachine + } + + // completeOnSendStateMachine is a generic state machine that transition + // into a comleted state immediately upon a command being sent (i.e. upon + // handleCommandSent() being called). + completeOnSendStateMachine struct { + *naiveCommandStateMachine + } + + modifyPropertiesCommandStateMachine struct { + *naiveCommandStateMachine + } + + // nexusOperationStateMachine is the state machine for the NexusOperation lifecycle. + // It may never transition to the started state if the operation completes synchronously. + // Valid transitions: + // commandStateCreated -> commandStateCommandSent + // commandStateCommandSent - (NexusOperationScheduled) -> commandStateInitiated + // commandStateInitiated - (NexusOperationStarted) -> commandStateStarted + // commandStateInitiated - (NexusOperation(Completed|Failed|Canceled|TimedOut)) -> commandStateCompleted + // commandStateStarted - (NexusOperation(Completed|Failed|Canceled|TimedOut)) -> commandStateCompleted + nexusOperationStateMachine struct { + *commandStateMachineBase + // Unique sequence number for identifying this machine SDK side. + seq int64 + // Event ID of the NexusOperationScheduled event for correlating progress events with this machine. + scheduledEventID int64 + attributes *commandpb.ScheduleNexusOperationCommandAttributes + // Instead of tracking cancelation as a state, we track it as a separate dimension with the request-cancel state + // machine. + cancelation *requestCancelNexusOperationStateMachine + } + + // requestCancelNexusOperationStateMachine is the state machine for the RequestCancelNexusOperation command. + // Valid transitions: + // commandStateCreated -> commandStateCommandSent + // commandStateCommandSent - (NexusOperationCancelRequested) -> commandStateCompleted + requestCancelNexusOperationStateMachine struct { + *commandStateMachineBase + attributes *commandpb.RequestCancelNexusOperationCommandAttributes + } + + versionMarker struct { + changeID string + searchAttrUpdated bool + } + + commandsHelper struct { + nextCommandEventID int64 + orderedCommands *list.List + commands map[commandID]*list.Element + + scheduledEventIDToActivityID map[int64]string + scheduledEventIDToCancellationID map[int64]string + scheduledEventIDToSignalID map[int64]string + versionMarkerLookup map[int64]versionMarker + + // A mapping of scheduled event ID to a sequence. + scheduledEventIDToNexusSeq map[int64]int64 + // A list containing all nexus operation machines that have not yet been assigned a scheduled event ID. + // Every new operation state machine is added to this list on creation and deleted once the scheduled event is + // seen or the operation was deleted before sending the command. + // This mechanism is based on Core SDK + // (https://github.com/temporalio/sdk-core/blob/16c7a33dc1aec8fafb33c9ad6f77569a3dacc8ea/core/src/worker/workflow/machines/workflow_machines.rs#L837). + nexusOperationsWithoutScheduledID *list.List + } + + // panic when command or message state machine is in illegal state + stateMachineIllegalStatePanic struct { + message string + } + + // Error returned when a child workflow with the same id already exists and hasn't completed + // and been removed from internal state. + childWorkflowExistsWithId struct { + id string + } +) + +const ( + commandStateCreated commandState = 0 + commandStateCommandSent commandState = 1 + commandStateCanceledBeforeInitiated commandState = 2 + commandStateInitiated commandState = 3 + commandStateStarted commandState = 4 + commandStateCanceledAfterInitiated commandState = 5 + commandStateCanceledAfterStarted commandState = 6 + commandStateCancellationCommandSent commandState = 7 + commandStateCompletedAfterCancellationCommandSent commandState = 8 + commandStateCompleted commandState = 9 + commandStateCanceledBeforeSent commandState = 10 + commandStateCancellationCommandAccepted commandState = 11 +) + +const ( + commandTypeActivity commandType = 0 + commandTypeChildWorkflow commandType = 1 + commandTypeCancellation commandType = 2 + commandTypeMarker commandType = 3 + commandTypeTimer commandType = 4 + commandTypeSignal commandType = 5 + commandTypeUpsertSearchAttributes commandType = 6 + commandTypeCancelTimer commandType = 7 + commandTypeRequestCancelActivityTask commandType = 8 + commandTypeAcceptWorkflowUpdate commandType = 9 + commandTypeCompleteWorkflowUpdate commandType = 10 + commandTypeModifyProperties commandType = 11 + commandTypeRejectWorkflowUpdate commandType = 12 + commandTypeProtocolMessage commandType = 13 + commandTypeNexusOperation commandType = 14 + commandTypeRequestCancelNexusOperation commandType = 15 +) + +const ( + eventCancel = "cancel" + eventCommandSent = "handleCommandSent" + eventInitiated = "handleInitiatedEvent" + eventInitiationFailed = "handleInitiationFailedEvent" + eventStarted = "handleStartedEvent" + eventCompletion = "handleCompletionEvent" + eventCancelInitiated = "handleCancelInitiatedEvent" + eventCancelFailed = "handleCancelFailedEvent" + eventCanceled = "handleCanceledEvent" + eventExternalWorkflowExecutionCancelRequested = "handleExternalWorkflowExecutionCancelRequested" +) + +const ( + sideEffectMarkerName = "SideEffect" + versionMarkerName = "Version" + localActivityMarkerName = "LocalActivity" + mutableSideEffectMarkerName = "MutableSideEffect" + + sideEffectMarkerIDName = "side-effect-id" + sideEffectMarkerDataName = "data" + versionMarkerChangeIDName = "change-id" + versionMarkerDataName = "version" + versionSearchAttributeUpdatedName = "version-search-attribute-updated" + localActivityMarkerDataName = "data" + localActivityResultName = "result" + mutableSideEffectCallCounterName = "mutable-side-effect-call-counter" +) + +func (d commandState) String() string { + switch d { + case commandStateCreated: + return "Created" + case commandStateCommandSent: + return "CommandSent" + case commandStateCanceledBeforeInitiated: + return "CanceledBeforeInitiated" + case commandStateInitiated: + return "Initiated" + case commandStateStarted: + return "Started" + case commandStateCanceledAfterInitiated: + return "CanceledAfterInitiated" + case commandStateCanceledAfterStarted: + return "CanceledAfterStarted" + case commandStateCancellationCommandSent: + return "CancellationCommandSent" + case commandStateCompletedAfterCancellationCommandSent: + return "CompletedAfterCancellationCommandSent" + case commandStateCompleted: + return "Completed" + case commandStateCanceledBeforeSent: + return "CanceledBeforeSent" + case commandStateCancellationCommandAccepted: + return "CancellationCommandAccepted" + default: + return fmt.Sprintf("Unknown: %d", int32(d)) + } +} + +func (d commandType) String() string { + switch d { + case commandTypeActivity: + return "Activity" + case commandTypeChildWorkflow: + return "ChildWorkflow" + case commandTypeCancellation: + return "Cancellation" + case commandTypeMarker: + return "Marker" + case commandTypeTimer: + return "Timer" + case commandTypeSignal: + return "Signal" + case commandTypeCancelTimer: + return "CancelTimer" + case commandTypeRequestCancelActivityTask: + return "RequestCancelActivityTask" + case commandTypeAcceptWorkflowUpdate: + return "AcceptWorkflowUpdate" + case commandTypeCompleteWorkflowUpdate: + return "CompleteWorkflowUpdate" + case commandTypeRejectWorkflowUpdate: + return "RejectWorkflowUpdate" + case commandTypeNexusOperation: + return "NexusOperation" + case commandTypeRequestCancelNexusOperation: + return "RequestCancelNexusOperation" + default: + return "Unknown" + } +} + +func (d commandID) String() string { + return fmt.Sprintf("CommandType: %v, ID: %v", d.commandType, d.id) +} + +func makeCommandID(commandType commandType, id string) commandID { + return commandID{commandType: commandType, id: id} +} + +func (h *commandsHelper) newCommandStateMachineBase(commandType commandType, id string) *commandStateMachineBase { + return &commandStateMachineBase{ + id: makeCommandID(commandType, id), + state: commandStateCreated, + history: []string{commandStateCreated.String()}, + helper: h, + } +} + +func (h *commandsHelper) newActivityCommandStateMachine( + scheduleID int64, + attributes *commandpb.ScheduleActivityTaskCommandAttributes, +) *activityCommandStateMachine { + base := h.newCommandStateMachineBase(commandTypeActivity, attributes.GetActivityId()) + return &activityCommandStateMachine{ + commandStateMachineBase: base, + scheduleID: scheduleID, + attributes: attributes, + } +} + +func (h *commandsHelper) newCancelActivityStateMachine(attributes *commandpb.RequestCancelActivityTaskCommandAttributes) *cancelActivityStateMachine { + base := h.newCommandStateMachineBase(commandTypeRequestCancelActivityTask, strconv.FormatInt(attributes.GetScheduledEventId(), 10)) + return &cancelActivityStateMachine{ + commandStateMachineBase: base, + attributes: attributes, + } +} + +func (h *commandsHelper) newNexusOperationStateMachine( + seq int64, + attributes *commandpb.ScheduleNexusOperationCommandAttributes, +) *nexusOperationStateMachine { + base := h.newCommandStateMachineBase(commandTypeNexusOperation, strconv.FormatInt(seq, 10)) + sm := &nexusOperationStateMachine{ + commandStateMachineBase: base, + attributes: attributes, + seq: seq, + // scheduledEventID will be assigned by the server when the corresponding event comes in. + } + h.nexusOperationsWithoutScheduledID.PushBack(sm) + return sm +} + +func (h *commandsHelper) newRequestCancelNexusOperationStateMachine(attributes *commandpb.RequestCancelNexusOperationCommandAttributes) *requestCancelNexusOperationStateMachine { + base := h.newCommandStateMachineBase(commandTypeRequestCancelNexusOperation, strconv.FormatInt(attributes.GetScheduledEventId(), 10)) + return &requestCancelNexusOperationStateMachine{ + commandStateMachineBase: base, + attributes: attributes, + } +} + +func (h *commandsHelper) newTimerCommandStateMachine( + attributes *commandpb.StartTimerCommandAttributes, + startMetadata *sdk.UserMetadata, +) *timerCommandStateMachine { + base := h.newCommandStateMachineBase(commandTypeTimer, attributes.GetTimerId()) + return &timerCommandStateMachine{ + commandStateMachineBase: base, + attributes: attributes, + startMetadata: startMetadata, + } +} + +func (h *commandsHelper) newCancelTimerCommandStateMachine(attributes *commandpb.CancelTimerCommandAttributes) *cancelTimerCommandStateMachine { + base := h.newCommandStateMachineBase(commandTypeCancelTimer, attributes.GetTimerId()) + return &cancelTimerCommandStateMachine{ + commandStateMachineBase: base, + attributes: attributes, + } +} + +func (h *commandsHelper) newChildWorkflowCommandStateMachine( + attributes *commandpb.StartChildWorkflowExecutionCommandAttributes, + startMetadata *sdk.UserMetadata, +) *childWorkflowCommandStateMachine { + base := h.newCommandStateMachineBase(commandTypeChildWorkflow, attributes.GetWorkflowId()) + return &childWorkflowCommandStateMachine{ + commandStateMachineBase: base, + attributes: attributes, + startMetadata: startMetadata, + } +} + +func (h *commandsHelper) newNaiveCommandStateMachine(commandType commandType, id string, command *commandpb.Command) *naiveCommandStateMachine { + base := h.newCommandStateMachineBase(commandType, id) + return &naiveCommandStateMachine{ + commandStateMachineBase: base, + command: command, + } +} + +func (h *commandsHelper) newMarkerCommandStateMachine(id string, attributes *commandpb.RecordMarkerCommandAttributes) *markerCommandStateMachine { + d := createNewCommand(enumspb.COMMAND_TYPE_RECORD_MARKER) + d.Attributes = &commandpb.Command_RecordMarkerCommandAttributes{RecordMarkerCommandAttributes: attributes} + return &markerCommandStateMachine{ + naiveCommandStateMachine: h.newNaiveCommandStateMachine(commandTypeMarker, id, d), + } +} + +func (h *commandsHelper) newCancelExternalWorkflowStateMachine(attributes *commandpb.RequestCancelExternalWorkflowExecutionCommandAttributes, cancellationID string) *cancelExternalWorkflowCommandStateMachine { + d := createNewCommand(enumspb.COMMAND_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION) + d.Attributes = &commandpb.Command_RequestCancelExternalWorkflowExecutionCommandAttributes{RequestCancelExternalWorkflowExecutionCommandAttributes: attributes} + return &cancelExternalWorkflowCommandStateMachine{ + naiveCommandStateMachine: h.newNaiveCommandStateMachine(commandTypeCancellation, cancellationID, d), + } +} + +func (h *commandsHelper) newSignalExternalWorkflowStateMachine(attributes *commandpb.SignalExternalWorkflowExecutionCommandAttributes, signalID string) *signalExternalWorkflowCommandStateMachine { + d := createNewCommand(enumspb.COMMAND_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION) + d.Attributes = &commandpb.Command_SignalExternalWorkflowExecutionCommandAttributes{SignalExternalWorkflowExecutionCommandAttributes: attributes} + return &signalExternalWorkflowCommandStateMachine{ + naiveCommandStateMachine: h.newNaiveCommandStateMachine(commandTypeSignal, signalID, d), + } +} + +func (h *commandsHelper) newUpsertSearchAttributesStateMachine(attributes *commandpb.UpsertWorkflowSearchAttributesCommandAttributes, upsertID string) *completeOnSendStateMachine { + d := createNewCommand(enumspb.COMMAND_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES) + d.Attributes = &commandpb.Command_UpsertWorkflowSearchAttributesCommandAttributes{UpsertWorkflowSearchAttributesCommandAttributes: attributes} + return &completeOnSendStateMachine{ + naiveCommandStateMachine: h.newNaiveCommandStateMachine(commandTypeUpsertSearchAttributes, upsertID, d), + } +} + +func (h *commandsHelper) newModifyPropertiesStateMachine( + attributes *commandpb.ModifyWorkflowPropertiesCommandAttributes, + changeID string, +) *modifyPropertiesCommandStateMachine { + d := createNewCommand(enumspb.COMMAND_TYPE_MODIFY_WORKFLOW_PROPERTIES) + d.Attributes = &commandpb.Command_ModifyWorkflowPropertiesCommandAttributes{ + ModifyWorkflowPropertiesCommandAttributes: attributes, + } + return &modifyPropertiesCommandStateMachine{ + naiveCommandStateMachine: h.newNaiveCommandStateMachine( + commandTypeModifyProperties, + changeID, + d, + ), + } +} + +func (d *commandStateMachineBase) getState() commandState { + return d.state +} + +func (d *commandStateMachineBase) getID() commandID { + return d.id +} + +func (d *commandStateMachineBase) isDone() bool { + return d.state == commandStateCompleted || d.state == commandStateCompletedAfterCancellationCommandSent +} + +func (d *commandStateMachineBase) setData(data interface{}) { + d.data = data +} + +func (d *commandStateMachineBase) getData() interface{} { + return d.data +} + +func (d *commandStateMachineBase) moveState(newState commandState, event string) { + d.history = append(d.history, event) + d.state = newState + d.history = append(d.history, newState.String()) + + if newState == commandStateCompleted { + if elem, ok := d.helper.commands[d.getID()]; ok { + d.helper.orderedCommands.Remove(elem) + delete(d.helper.commands, d.getID()) + } + } +} + +func (d stateMachineIllegalStatePanic) String() string { + return d.message +} + +func panicIllegalState(message string) { + panic(stateMachineIllegalStatePanic{message: message}) +} + +func (d *commandStateMachineBase) failStateTransition(event string) { + // this is when we detect illegal state transition, likely due to ill history sequence or nondeterministic workflow code + panicIllegalState(fmt.Sprintf("[TMPRL1100] invalid state transition: attempt to %v, %v", event, d)) +} + +func (d *commandStateMachineBase) handleCommandSent() { + switch d.state { + case commandStateCreated: + d.moveState(commandStateCommandSent, eventCommandSent) + } +} + +func (d *commandStateMachineBase) cancel() { + switch d.state { + case commandStateCompleted, commandStateCompletedAfterCancellationCommandSent: + // No op. This is legit. People could cancel context after timer/activity is done. + case commandStateCreated: + d.moveState(commandStateCanceledBeforeSent, eventCancel) + case commandStateCommandSent: + d.moveState(commandStateCancellationCommandSent, eventCancel) + case commandStateInitiated: + d.moveState(commandStateCanceledAfterInitiated, eventCancel) + default: + d.failStateTransition(eventCancel) + } +} + +func (d *commandStateMachineBase) handleInitiatedEvent() { + switch d.state { + case commandStateCommandSent: + d.moveState(commandStateInitiated, eventInitiated) + case commandStateCanceledBeforeInitiated, commandStateCanceledBeforeSent, commandStateCancellationCommandSent: + d.moveState(commandStateCanceledAfterInitiated, eventInitiated) + default: + d.failStateTransition(eventInitiated) + } +} + +func (d *commandStateMachineBase) handleInitiationFailedEvent() { + switch d.state { + case commandStateInitiated, commandStateCommandSent, commandStateCanceledBeforeInitiated, commandStateCancellationCommandSent: + d.moveState(commandStateCompleted, eventInitiationFailed) + default: + d.failStateTransition(eventInitiationFailed) + } +} + +func (d *commandStateMachineBase) handleStartedEvent() { + d.history = append(d.history, eventStarted) +} + +func (d *commandStateMachineBase) handleCompletionEvent() { + switch d.state { + case commandStateCanceledAfterInitiated, commandStateInitiated: + d.moveState(commandStateCompleted, eventCompletion) + case commandStateCancellationCommandSent: + d.moveState(commandStateCompletedAfterCancellationCommandSent, eventCompletion) + default: + d.failStateTransition(eventCompletion) + } +} + +func (d *commandStateMachineBase) handleCancelInitiatedEvent() { + d.history = append(d.history, eventCancelInitiated) + switch d.state { + case commandStateCancellationCommandSent, commandStateCanceledAfterInitiated: + // No state change + default: + d.failStateTransition(eventCancelInitiated) + } +} + +func (d *commandStateMachineBase) handleCancelFailedEvent() { + switch d.state { + case commandStateCompletedAfterCancellationCommandSent: + d.moveState(commandStateCompleted, eventCancelFailed) + default: + d.failStateTransition(eventCancelFailed) + } +} + +func (d *commandStateMachineBase) handleCanceledEvent() { + switch d.state { + case commandStateCancellationCommandSent, commandStateCanceledAfterInitiated, commandStateCanceledAfterStarted, commandStateCancellationCommandAccepted: + d.moveState(commandStateCompleted, eventCanceled) + default: + d.failStateTransition(eventCanceled) + } +} + +func (d *commandStateMachineBase) String() string { + return fmt.Sprintf("%v, state=%v, isDone()=%v, history=%v", + d.id, d.state, d.isDone(), d.history) +} + +func (d *activityCommandStateMachine) getCommand() *commandpb.Command { + switch d.state { + case commandStateCreated, commandStateCanceledBeforeSent: + command := createNewCommand(enumspb.COMMAND_TYPE_SCHEDULE_ACTIVITY_TASK) + command.Attributes = &commandpb.Command_ScheduleActivityTaskCommandAttributes{ScheduleActivityTaskCommandAttributes: d.attributes} + return command + default: + return nil + } +} + +func (d *activityCommandStateMachine) handleCommandSent() { + switch d.state { + case commandStateCanceledAfterInitiated: + d.moveState(commandStateCancellationCommandSent, eventCommandSent) + default: + d.commandStateMachineBase.handleCommandSent() + } +} + +func (d *activityCommandStateMachine) handleCancelFailedEvent() { + // Request to cancel activity now results in either activity completion, failed, timedout, or canceled + // Request to cancel itself can never fail and invalid RequestCancelActivity commands results in the + // entire command being failed. + d.failStateTransition(eventCancelFailed) +} + +func (d *activityCommandStateMachine) cancel() { + switch d.state { + case commandStateCreated, commandStateCommandSent, commandStateInitiated: + attribs := &commandpb.RequestCancelActivityTaskCommandAttributes{ + ScheduledEventId: d.scheduleID, + } + cancelCmd := d.helper.newCancelActivityStateMachine(attribs) + d.helper.addCommand(cancelCmd) + // We also mark the schedule command as not eager if we haven't sent it yet. + // Server behavior differs on eager vs non-eager when scheduling and + // cancelling during the same task completion. If it has not been sent this + // means we are cancelling at the same time as scheduling which is not + // properly supported for eager activities. + if d.state != commandStateCommandSent { + d.attributes.RequestEagerExecution = false + } + } + + d.commandStateMachineBase.cancel() +} + +func (d *timerCommandStateMachine) cancel() { + switch d.state { + case commandStateCreated, commandStateCommandSent, commandStateInitiated: + attribs := &commandpb.CancelTimerCommandAttributes{ + TimerId: d.attributes.TimerId, + } + cancelCmd := d.helper.newCancelTimerCommandStateMachine(attribs) + d.helper.addCommand(cancelCmd) + } + + d.commandStateMachineBase.cancel() +} + +func (d *timerCommandStateMachine) isDone() bool { + return d.state == commandStateCompleted +} + +func (d *timerCommandStateMachine) handleCommandSent() { + switch d.state { + case commandStateCanceledAfterInitiated: + d.moveState(commandStateCancellationCommandSent, eventCommandSent) + default: + d.commandStateMachineBase.handleCommandSent() + } +} + +func (d *cancelActivityStateMachine) getCommand() *commandpb.Command { + switch d.state { + case commandStateCreated: + command := createNewCommand(enumspb.COMMAND_TYPE_REQUEST_CANCEL_ACTIVITY_TASK) + command.Attributes = &commandpb.Command_RequestCancelActivityTaskCommandAttributes{RequestCancelActivityTaskCommandAttributes: d.attributes} + return command + default: + return nil + } +} + +func (d *timerCommandStateMachine) getCommand() *commandpb.Command { + switch d.state { + case commandStateCreated, commandStateCanceledBeforeSent: + command := createNewCommand(enumspb.COMMAND_TYPE_START_TIMER) + command.Attributes = &commandpb.Command_StartTimerCommandAttributes{StartTimerCommandAttributes: d.attributes} + command.UserMetadata = d.startMetadata + return command + default: + return nil + } +} + +func (d *cancelTimerCommandStateMachine) getCommand() *commandpb.Command { + switch d.state { + case commandStateCreated: + command := createNewCommand(enumspb.COMMAND_TYPE_CANCEL_TIMER) + command.Attributes = &commandpb.Command_CancelTimerCommandAttributes{CancelTimerCommandAttributes: d.attributes} + return command + default: + return nil + } +} + +func (d *childWorkflowCommandStateMachine) getCommand() *commandpb.Command { + switch d.state { + case commandStateCreated: + command := createNewCommand(enumspb.COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION) + command.Attributes = &commandpb.Command_StartChildWorkflowExecutionCommandAttributes{StartChildWorkflowExecutionCommandAttributes: d.attributes} + command.UserMetadata = d.startMetadata + return command + case commandStateCanceledAfterStarted: + command := createNewCommand(enumspb.COMMAND_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION) + command.Attributes = &commandpb.Command_RequestCancelExternalWorkflowExecutionCommandAttributes{RequestCancelExternalWorkflowExecutionCommandAttributes: &commandpb.RequestCancelExternalWorkflowExecutionCommandAttributes{ + Namespace: d.attributes.Namespace, + WorkflowId: d.attributes.WorkflowId, + ChildWorkflowOnly: true, + }} + return command + default: + return nil + } +} + +func (d *childWorkflowCommandStateMachine) handleCommandSent() { + switch d.state { + case commandStateCanceledAfterStarted: + d.moveState(commandStateCancellationCommandSent, eventCommandSent) + default: + d.commandStateMachineBase.handleCommandSent() + } +} + +func (d *childWorkflowCommandStateMachine) handleStartedEvent() { + switch d.state { + case commandStateInitiated: + d.moveState(commandStateStarted, eventStarted) + case commandStateCanceledAfterInitiated: + d.moveState(commandStateCanceledAfterStarted, eventStarted) + default: + d.commandStateMachineBase.handleStartedEvent() + } +} + +func (d *childWorkflowCommandStateMachine) handleInitiatedEvent() { + switch d.state { + case commandStateCancellationCommandSent: + d.failStateTransition(eventInitiated) + default: + d.commandStateMachineBase.handleInitiatedEvent() + } +} + +func (d *childWorkflowCommandStateMachine) handleCancelFailedEvent() { + switch d.state { + case commandStateCancellationCommandSent: + d.moveState(commandStateStarted, eventCancelFailed) + default: + d.commandStateMachineBase.handleCancelFailedEvent() + } +} + +func (d *childWorkflowCommandStateMachine) cancel() { + switch d.state { + case commandStateStarted: + d.moveState(commandStateCanceledAfterStarted, eventCancel) + // A child workflow may be canceled _after_ something like an activity start + // happens inside a simulated goroutine. However, since the state of the + // entire child workflow is recorded based on when it started not when it + // was canceled, we have to move it to the end once canceled to keep the + // expected commands in order of when they actually occurred. + d.helper.moveCommandToBack(d) + d.helper.incrementNextCommandEventID() + default: + d.commandStateMachineBase.cancel() + } +} + +func (d *childWorkflowCommandStateMachine) handleCanceledEvent() { + switch d.state { + case commandStateStarted: + d.moveState(commandStateCompleted, eventCanceled) + case commandStateCancellationCommandSent: + // We've sent the command but haven't seen the server accept the cancellation. We must ensure this command hangs + // around, because it is possible for the child workflow to be canceled before we've seen the event. + d.moveState(commandStateCompletedAfterCancellationCommandSent, eventCanceled) + default: + d.commandStateMachineBase.handleCanceledEvent() + } +} + +func (d *childWorkflowCommandStateMachine) handleCompletionEvent() { + switch d.state { + case commandStateStarted, commandStateCanceledAfterStarted, + commandStateCompletedAfterCancellationCommandSent, commandStateCancellationCommandAccepted: + d.moveState(commandStateCompleted, eventCompletion) + default: + d.commandStateMachineBase.handleCompletionEvent() + } +} + +func (d *childWorkflowCommandStateMachine) handleExternalWorkflowExecutionCancelRequested() { + if d.getState() == commandStateCompletedAfterCancellationCommandSent { + // Now we're really done. + d.handleCompletionEvent() + } else { + // We should be in the cancellation command sent stage - new state to indicate we have seen the cancel accepted + d.moveState(commandStateCancellationCommandAccepted, eventExternalWorkflowExecutionCancelRequested) + } +} + +func (d *naiveCommandStateMachine) getCommand() *commandpb.Command { + switch d.state { + case commandStateCreated: + return d.command + default: + return nil + } +} + +func (d *naiveCommandStateMachine) cancel() { + panic("unsupported operation") +} + +func (d *naiveCommandStateMachine) handleCompletionEvent() { + panic("unsupported operation") +} + +func (d *naiveCommandStateMachine) handleInitiatedEvent() { + panic("unsupported operation") +} + +func (d *naiveCommandStateMachine) handleInitiationFailedEvent() { + panic("unsupported operation") +} + +func (d *naiveCommandStateMachine) handleStartedEvent() { + panic("unsupported operation") +} + +func (d *naiveCommandStateMachine) handleCanceledEvent() { + panic("unsupported operation") +} + +func (d *naiveCommandStateMachine) handleCancelFailedEvent() { + panic("unsupported operation") +} + +func (d *naiveCommandStateMachine) handleCancelInitiatedEvent() { + panic("unsupported operation") +} + +func (d *cancelExternalWorkflowCommandStateMachine) handleInitiatedEvent() { + switch d.state { + case commandStateCommandSent: + d.moveState(commandStateInitiated, eventInitiated) + default: + d.failStateTransition(eventInitiated) + } +} + +func (d *cancelExternalWorkflowCommandStateMachine) handleCompletionEvent() { + switch d.state { + case commandStateInitiated: + d.moveState(commandStateCompleted, eventCompletion) + default: + d.failStateTransition(eventCompletion) + } +} + +func (d *signalExternalWorkflowCommandStateMachine) handleInitiatedEvent() { + switch d.state { + case commandStateCommandSent: + d.moveState(commandStateInitiated, eventInitiated) + default: + d.failStateTransition(eventInitiated) + } +} + +func (d *signalExternalWorkflowCommandStateMachine) handleCompletionEvent() { + switch d.state { + case commandStateInitiated: + d.moveState(commandStateCompleted, eventCompletion) + default: + d.failStateTransition(eventCompletion) + } +} + +func (d *markerCommandStateMachine) handleCommandSent() { + // Marker command state machine is considered as completed once command is sent. + // For SideEffect/Version markers, when the history event is applied, there is no marker command state machine yet + // because we preload those marker events. + // For local activity, when we apply the history event, we use it to create the marker state machine, there is no + // other event to drive it to completed state. + switch d.state { + case commandStateCreated: + d.moveState(commandStateCompleted, eventCommandSent) + } +} + +func (d *completeOnSendStateMachine) handleCommandSent() { + // This command is considered as completed once command is sent. + switch d.state { + case commandStateCreated: + d.moveState(commandStateCompleted, eventCommandSent) + } +} + +func (d *modifyPropertiesCommandStateMachine) handleCommandSent() { + // This command is considered as completed once command is sent. + switch d.state { + case commandStateCreated: + d.moveState(commandStateCompleted, eventCommandSent) + } +} + +func (sm *nexusOperationStateMachine) getCommand() *commandpb.Command { + if sm.state == commandStateCreated && sm.cancelation == nil { + // Only create the command in this state unlike other machines that also create it if canceled before sent. + return &commandpb.Command{ + CommandType: enumspb.COMMAND_TYPE_SCHEDULE_NEXUS_OPERATION, + Attributes: &commandpb.Command_ScheduleNexusOperationCommandAttributes{ + ScheduleNexusOperationCommandAttributes: sm.attributes, + }, + } + } + return nil +} + +func (sm *nexusOperationStateMachine) handleStartedEvent() { + switch sm.state { + case commandStateInitiated: + sm.moveState(commandStateStarted, eventStarted) + default: + sm.failStateTransition(eventStarted) + } +} + +func (sm *nexusOperationStateMachine) handleCompletionEvent() { + switch sm.state { + case commandStateInitiated, + commandStateStarted: + sm.moveState(commandStateCompleted, eventCompletion) + default: + sm.failStateTransition(eventStarted) + } +} + +func (sm *nexusOperationStateMachine) cancel() { + // Already canceled or already completed. + if sm.cancelation != nil || sm.state == commandStateCompleted { + return + } + + attribs := &commandpb.RequestCancelNexusOperationCommandAttributes{ + ScheduledEventId: sm.scheduledEventID, + } + cancelCmd := sm.helper.newRequestCancelNexusOperationStateMachine(attribs) + sm.cancelation = cancelCmd + sm.helper.addCommand(cancelCmd) + + // No need to actually send the cancelation, mark the state machine as completed. + if sm.state == commandStateCreated { + cancelCmd.handleCompletionEvent() + } +} + +func (d *requestCancelNexusOperationStateMachine) getCommand() *commandpb.Command { + switch d.state { + case commandStateCreated: + command := createNewCommand(enumspb.COMMAND_TYPE_REQUEST_CANCEL_NEXUS_OPERATION) + command.Attributes = &commandpb.Command_RequestCancelNexusOperationCommandAttributes{RequestCancelNexusOperationCommandAttributes: d.attributes} + return command + default: + return nil + } +} + +func (d *requestCancelNexusOperationStateMachine) handleCompletionEvent() { + if d.state != commandStateCommandSent && d.state != commandStateCreated { + d.failStateTransition(eventCompletion) + return + } + d.moveState(commandStateCompleted, eventCompletion) +} + +func newCommandsHelper() *commandsHelper { + return &commandsHelper{ + orderedCommands: list.New(), + commands: make(map[commandID]*list.Element), + + scheduledEventIDToActivityID: make(map[int64]string), + scheduledEventIDToCancellationID: make(map[int64]string), + scheduledEventIDToSignalID: make(map[int64]string), + versionMarkerLookup: make(map[int64]versionMarker), + scheduledEventIDToNexusSeq: make(map[int64]int64), + nexusOperationsWithoutScheduledID: list.New(), + } +} + +func (h *commandsHelper) incrementNextCommandEventID() { + h.nextCommandEventID++ +} + +func (h *commandsHelper) setCurrentWorkflowTaskStartedEventID(workflowTaskStartedEventID int64) { + // Server always processes the commands in the same order it is generated by client and each command results in + // corresponding history event after processing. So we can use workflow task started event id + 2 as the offset as + // workflow task completed event is always the first event in the workflow task followed by events generated from + // commands. This allows client sdk to deterministically predict history event ids generated by processing of the + // command. It is possible, notably during workflow cancellation, that commands are generated before the workflow + // task started event is processed. In this case we need to adjust the nextCommandEventID to account for these unsent + // commands.git + var uncountedCommands int64 + for curr := h.orderedCommands.Front(); curr != nil; { + d := curr.Value.(commandStateMachine) + command := d.getCommand() + if command != nil { + uncountedCommands += 1 + } + curr = curr.Next() + } + + h.nextCommandEventID = workflowTaskStartedEventID + 2 + uncountedCommands +} + +func (h *commandsHelper) getNextID() int64 { + // First check if we have a GetVersion marker in the lookup map + h.incrementNextCommandEventIDIfVersionMarker() + if h.nextCommandEventID == 0 { + panic("Attempt to generate a command before processing WorkflowTaskStarted event") + } + return h.nextCommandEventID +} + +func (h *commandsHelper) incrementNextCommandEventIDIfVersionMarker() { + marker, ok := h.versionMarkerLookup[h.nextCommandEventID] + for ok { + // Remove the marker from the lookup map and increment nextCommandEventID by 2 because call to GetVersion + // results in 1 or 2 events in the history. One is GetVersion marker event for changeID and change version, other + // is UpsertSearchableAttributes to keep track of executions using particular version of code. + delete(h.versionMarkerLookup, h.nextCommandEventID) + h.incrementNextCommandEventID() + // UpsertSearchableAttributes may not have been written if the search attribute was too large. + if marker.searchAttrUpdated { + h.incrementNextCommandEventID() + } + marker, ok = h.versionMarkerLookup[h.nextCommandEventID] + } +} + +func (h *commandsHelper) getCommand(id commandID) commandStateMachine { + command, ok := h.commands[id] + if !ok { + panicMsg := fmt.Sprintf("[TMPRL1100] unknown command %v, possible causes are nondeterministic workflow definition code"+ + " or incompatible change in the workflow definition", id) + panicIllegalState(panicMsg) + } + return command.Value.(commandStateMachine) +} + +func (h *commandsHelper) addCommand(command commandStateMachine) { + if _, ok := h.commands[command.getID()]; ok { + panicMsg := fmt.Sprintf("[TMPRL1100] adding duplicate command %v", command) + panicIllegalState(panicMsg) + } + element := h.orderedCommands.PushBack(command) + h.commands[command.getID()] = element + + // Every time new command is added increment the counter used for generating ID + h.incrementNextCommandEventIDIfVersionMarker() + h.incrementNextCommandEventID() +} + +// This really should not exist, but is unavoidable without totally redesigning the Go SDK to avoid +// doing event number counting. EX: Because a workflow execution cancel requested event calls a callback +// on timers that immediately cancels them, we will queue up a cancel timer command even though that timer firing +// might be in the same workflow task. In practice this only seems to happen during unhandled command events. +func (h *commandsHelper) removeCancelOfResolvedCommand(commandID commandID) { + // Ensure this isn't misused for non-cancel commands + if commandID.commandType != commandTypeCancelTimer && commandID.commandType != commandTypeRequestCancelActivityTask { + panic("removeCancelOfResolvedCommand should only be called for cancel timer / activity") + } + orderedCmdEl, ok := h.commands[commandID] + if ok { + delete(h.commands, commandID) + _ = h.orderedCommands.Remove(orderedCmdEl) + } +} + +func (h *commandsHelper) moveCommandToBack(command commandStateMachine) { + elem := h.commands[command.getID()] + if elem == nil { + panicIllegalState(fmt.Sprintf("[TMPRL1100] moving command not present %v", command)) + } + h.orderedCommands.Remove(elem) + h.commands[command.getID()] = h.orderedCommands.PushBack(command) +} + +func (h *commandsHelper) scheduleActivityTask( + scheduleID int64, + attributes *commandpb.ScheduleActivityTaskCommandAttributes, +) commandStateMachine { + h.scheduledEventIDToActivityID[scheduleID] = attributes.GetActivityId() + command := h.newActivityCommandStateMachine(scheduleID, attributes) + h.addCommand(command) + return command +} + +func (h *commandsHelper) requestCancelActivityTask(activityID string) commandStateMachine { + id := makeCommandID(commandTypeActivity, activityID) + command := h.getCommand(id) + command.cancel() + return command +} + +func (h *commandsHelper) handleActivityTaskClosed(activityID string, scheduledEventID int64) commandStateMachine { + command := h.getCommand(makeCommandID(commandTypeActivity, activityID)) + // If, for whatever reason, we were going to send an activity cancel request, don't do that anymore + // since we already know the activity is resolved. + possibleCancelID := makeCommandID(commandTypeRequestCancelActivityTask, activityID) + h.removeCancelOfResolvedCommand(possibleCancelID) + command.handleCompletionEvent() + delete(h.scheduledEventIDToActivityID, scheduledEventID) + return command +} + +func (h *commandsHelper) handleActivityTaskScheduled(activityID string, scheduledEventID int64) { + if _, ok := h.scheduledEventIDToActivityID[scheduledEventID]; !ok { + panicMsg := fmt.Sprintf("[TMPRL1100] lookup failed for scheduledEventID to activityID: scheduleEventID: %v, activityID: %v", + scheduledEventID, activityID) + panicIllegalState(panicMsg) + } + + command := h.getCommand(makeCommandID(commandTypeActivity, activityID)) + command.handleInitiatedEvent() +} + +func (h *commandsHelper) handleActivityTaskCancelRequested(scheduledEventID int64) { + activityID, ok := h.scheduledEventIDToActivityID[scheduledEventID] + if !ok { + panicIllegalState(fmt.Sprintf("[TMPRL1100] unable to find activityID for the scheduledEventID: %v", scheduledEventID)) + } + command := h.getCommand(makeCommandID(commandTypeActivity, activityID)) + command.handleCancelInitiatedEvent() +} + +func (h *commandsHelper) handleActivityTaskCanceled(activityID string, scheduledEventID int64) commandStateMachine { + command := h.getCommand(makeCommandID(commandTypeActivity, activityID)) + command.handleCanceledEvent() + delete(h.scheduledEventIDToActivityID, scheduledEventID) + return command +} + +func (h *commandsHelper) getActivityAndScheduledEventIDs(event *historypb.HistoryEvent) (string, int64) { + var scheduledEventID int64 = -1 + switch event.GetEventType() { + case enumspb.EVENT_TYPE_ACTIVITY_TASK_CANCELED: + scheduledEventID = event.GetActivityTaskCanceledEventAttributes().GetScheduledEventId() + case enumspb.EVENT_TYPE_ACTIVITY_TASK_COMPLETED: + scheduledEventID = event.GetActivityTaskCompletedEventAttributes().GetScheduledEventId() + case enumspb.EVENT_TYPE_ACTIVITY_TASK_FAILED: + scheduledEventID = event.GetActivityTaskFailedEventAttributes().GetScheduledEventId() + case enumspb.EVENT_TYPE_ACTIVITY_TASK_TIMED_OUT: + scheduledEventID = event.GetActivityTaskTimedOutEventAttributes().GetScheduledEventId() + default: + panicIllegalState(fmt.Sprintf("[TMPRL1100] unexpected event type: %v", event.GetEventType())) + } + + activityID, ok := h.scheduledEventIDToActivityID[scheduledEventID] + if !ok { + panicIllegalState(fmt.Sprintf("[TMPRL1100] unable to find activityID for the event: %v", util.HistoryEventToString(event))) + } + return activityID, scheduledEventID +} + +func (h *commandsHelper) scheduleNexusOperation( + seq int64, + attributes *commandpb.ScheduleNexusOperationCommandAttributes, +) *nexusOperationStateMachine { + command := h.newNexusOperationStateMachine(seq, attributes) + h.addCommand(command) + return command +} + +func (h *commandsHelper) handleNexusOperationScheduled(event *historypb.HistoryEvent) { + elem := h.nexusOperationsWithoutScheduledID.Front() + if elem == nil { + panicIllegalState(fmt.Sprintf("[TMPRL1100] unable to find nexus operation state machine for event: %v", util.HistoryEventToString(event))) + } + command := h.nexusOperationsWithoutScheduledID.Remove(elem).(*nexusOperationStateMachine) + + command.scheduledEventID = event.EventId + h.scheduledEventIDToNexusSeq[event.EventId] = command.seq + command.handleInitiatedEvent() +} + +func (h *commandsHelper) handleNexusOperationStarted(scheduledEventID int64) commandStateMachine { + seq, ok := h.scheduledEventIDToNexusSeq[scheduledEventID] + if !ok { + panicIllegalState(fmt.Sprintf("[TMPRL1100] unable to find nexus operation state machine for event ID: %v", scheduledEventID)) + } + command := h.getCommand(makeCommandID(commandTypeNexusOperation, strconv.FormatInt(seq, 10))) + command.handleStartedEvent() + return command +} + +func (h *commandsHelper) handleNexusOperationCompleted(scheduledEventID int64) commandStateMachine { + seq, ok := h.scheduledEventIDToNexusSeq[scheduledEventID] + if !ok { + panicIllegalState(fmt.Sprintf("[TMPRL1100] unable to find nexus operation state machine for event ID: %v", scheduledEventID)) + } + // We don't need this anymore, the state will not transition after completion. + delete(h.scheduledEventIDToNexusSeq, scheduledEventID) + command := h.getCommand(makeCommandID(commandTypeNexusOperation, strconv.FormatInt(seq, 10))) + command.handleCompletionEvent() + return command +} + +func (h *commandsHelper) handleNexusOperationCancelRequested(scheduledEventID int64) { + command := h.getCommand(makeCommandID(commandTypeRequestCancelNexusOperation, strconv.FormatInt(scheduledEventID, 10))) + command.handleCompletionEvent() +} + +func (h *commandsHelper) requestCancelNexusOperation(seq int64) commandStateMachine { + command := h.getCommand(makeCommandID(commandTypeNexusOperation, strconv.FormatInt(seq, 10))) + command.cancel() + // If we haven't sent the command yet, ensure that it doesn't get mapped to the wrong scheduledEventID. + if command.getState() != commandStateCanceledBeforeSent { + return command + } + for elem := h.nexusOperationsWithoutScheduledID.Front(); elem != nil; elem = elem.Next() { + sm := elem.Value.(*nexusOperationStateMachine) + if sm.seq == seq { + h.nexusOperationsWithoutScheduledID.Remove(elem) + break + } + } + return command +} + +func (h *commandsHelper) recordVersionMarker(changeID string, version Version, dc converter.DataConverter, searchAttributeWasUpdated bool) commandStateMachine { + markerID := fmt.Sprintf("%v_%v", versionMarkerName, changeID) + + changeIDPayload, err := dc.ToPayloads(changeID) + if err != nil { + panic(err) + } + + versionPayload, err := dc.ToPayloads(version) + if err != nil { + panic(err) + } + + recordMarker := &commandpb.RecordMarkerCommandAttributes{ + MarkerName: versionMarkerName, + Details: map[string]*commonpb.Payloads{ + versionMarkerChangeIDName: changeIDPayload, + versionMarkerDataName: versionPayload, + }, + } + + if !searchAttributeWasUpdated { + searchAttributeWasUpdatedPayload, err := dc.ToPayloads(searchAttributeWasUpdated) + if err != nil { + panic(err) + } + recordMarker.Details[versionSearchAttributeUpdatedName] = searchAttributeWasUpdatedPayload + } + + command := h.newMarkerCommandStateMachine(markerID, recordMarker) + h.addCommand(command) + return command +} + +func (h *commandsHelper) handleVersionMarker(eventID int64, changeID string, searchAttrUpdated bool) { + if _, ok := h.versionMarkerLookup[eventID]; ok { + panicMsg := fmt.Sprintf("[TMPRL1100] marker event already exists for eventID in lookup: eventID: %v, changeID: %v", + eventID, changeID) + panicIllegalState(panicMsg) + } + + // During processing of a workflow task we reorder all GetVersion markers and process them first. + // Keep track of all GetVersion marker events during the processing of workflow task so we can + // generate correct eventIDs for other events during replay. + h.versionMarkerLookup[eventID] = versionMarker{ + changeID: changeID, + searchAttrUpdated: searchAttrUpdated, + } +} + +func (h *commandsHelper) recordSideEffectMarker(sideEffectID int64, data *commonpb.Payloads, dc converter.DataConverter) commandStateMachine { + markerID := fmt.Sprintf("%v_%v", sideEffectMarkerName, sideEffectID) + sideEffectIDPayload, err := dc.ToPayloads(sideEffectID) + if err != nil { + panic(err) + } + + attributes := &commandpb.RecordMarkerCommandAttributes{ + MarkerName: sideEffectMarkerName, + Details: map[string]*commonpb.Payloads{ + sideEffectMarkerIDName: sideEffectIDPayload, + sideEffectMarkerDataName: data, + }, + } + command := h.newMarkerCommandStateMachine(markerID, attributes) + h.addCommand(command) + return command +} + +func (h *commandsHelper) recordLocalActivityMarker(activityID string, details map[string]*commonpb.Payloads, failure *failurepb.Failure) commandStateMachine { + markerID := fmt.Sprintf("%v_%v", localActivityMarkerName, activityID) + attributes := &commandpb.RecordMarkerCommandAttributes{ + MarkerName: localActivityMarkerName, + Failure: failure, + Details: details, + } + command := h.newMarkerCommandStateMachine(markerID, attributes) + // LocalActivity marker is added only when it completes and schedule logic never relies on GenerateSequence to + // create a unique activity id like in the case of ExecuteActivity. This causes the problem as we only perform + // the check to increment counter to account for GetVersion special handling as part of it. This will result + // in wrong IDs to be generated if there is GetVersion call before local activities. Explicitly calling getNextID + // to correctly incrementing counter before adding the command. + h.getNextID() + h.addCommand(command) + return command +} + +func (h *commandsHelper) recordMutableSideEffectMarker(mutableSideEffectID string, callCountHint int, data *commonpb.Payloads, dc converter.DataConverter) commandStateMachine { + // In order to avoid duplicate marker IDs, we must append the counter to the + // user-provided ID + mutableSideEffectID = fmt.Sprintf("%v_%v", mutableSideEffectID, h.getNextID()) + markerID := fmt.Sprintf("%v_%v", mutableSideEffectMarkerName, mutableSideEffectID) + + mutableSideEffectIDPayload, err := dc.ToPayloads(mutableSideEffectID) + if err != nil { + panic(err) + } + + mutableSideEffectCounterPayload, err := dc.ToPayloads(callCountHint) + if err != nil { + panic(err) + } + + attributes := &commandpb.RecordMarkerCommandAttributes{ + MarkerName: mutableSideEffectMarkerName, + Details: map[string]*commonpb.Payloads{ + sideEffectMarkerIDName: mutableSideEffectIDPayload, + sideEffectMarkerDataName: data, + mutableSideEffectCallCounterName: mutableSideEffectCounterPayload, + }, + } + command := h.newMarkerCommandStateMachine(markerID, attributes) + h.addCommand(command) + return command +} + +// startChildWorkflowExecution can return an error in the event that there is already a child wf +// with the same ID which exists as a command in memory. Other SDKs actually will send this command +// to server, and have it reject it - but here the command ID is exactly equal to the child's wf ID, +// and changing that without potentially blowing up backwards compatability is difficult. So we +// return the error eagerly locally, which is at least an improvement on panicking. +func (h *commandsHelper) startChildWorkflowExecution( + attributes *commandpb.StartChildWorkflowExecutionCommandAttributes, + startMetadata *sdk.UserMetadata, +) (commandStateMachine, error) { + command := h.newChildWorkflowCommandStateMachine(attributes, startMetadata) + if h.commands[command.getID()] != nil { + return nil, &childWorkflowExistsWithId{id: attributes.WorkflowId} + } + h.addCommand(command) + return command, nil +} + +func (h *commandsHelper) handleStartChildWorkflowExecutionInitiated(workflowID string) { + command := h.getCommand(makeCommandID(commandTypeChildWorkflow, workflowID)) + command.handleInitiatedEvent() +} + +func (h *commandsHelper) handleStartChildWorkflowExecutionFailed(workflowID string) commandStateMachine { + command := h.getCommand(makeCommandID(commandTypeChildWorkflow, workflowID)) + command.handleInitiationFailedEvent() + return command +} + +func (h *commandsHelper) requestCancelExternalWorkflowExecution(namespace, workflowID, runID string, cancellationID string, childWorkflowOnly bool) commandStateMachine { + if childWorkflowOnly { + // For cancellation of child workflow only, we do not use cancellation ID + // since the child workflow cancellation go through the existing child workflow + // state machine, and we use workflow ID as identifier + // we also do not use run ID, since child workflow can do continue-as-new + // which will have different run ID + // there will be server side validation that target workflow is child workflow + + // sanity check that cancellation ID is not set + if len(cancellationID) != 0 { + panic("cancellation on child workflow should not use cancellation ID") + } + // sanity check that run ID is not set + if len(runID) != 0 { + panic("cancellation on child workflow should not use run ID") + } + // targeting child workflow + command := h.getCommand(makeCommandID(commandTypeChildWorkflow, workflowID)) + command.cancel() + return command + } + + // For cancellation of external workflow, we have to use cancellation ID + // to identify different cancellation request (command) / response (history event) + // client can also use this code path to cancel its own child workflow, however, there will + // be no server side validation that target workflow is the child + + // sanity check that cancellation ID is set + if len(cancellationID) == 0 { + panic("cancellation on external workflow should use cancellation ID") + } + attributes := &commandpb.RequestCancelExternalWorkflowExecutionCommandAttributes{ + Namespace: namespace, + WorkflowId: workflowID, + RunId: runID, + Control: cancellationID, + ChildWorkflowOnly: false, + } + command := h.newCancelExternalWorkflowStateMachine(attributes, cancellationID) + h.addCommand(command) + + return command +} + +func (h *commandsHelper) handleRequestCancelExternalWorkflowExecutionInitiated(initiatedeventID int64, workflowID, cancellationID string) { + if h.isCancelExternalWorkflowEventForChildWorkflow(cancellationID) { + // this is cancellation for child workflow only + command := h.getCommand(makeCommandID(commandTypeChildWorkflow, workflowID)) + command.handleCancelInitiatedEvent() + } else { + // this is cancellation for external workflow + h.scheduledEventIDToCancellationID[initiatedeventID] = cancellationID + command := h.getCommand(makeCommandID(commandTypeCancellation, cancellationID)) + command.handleInitiatedEvent() + } +} + +func (h *commandsHelper) handleExternalWorkflowExecutionCancelRequested(initiatedeventID int64, workflowID string) (bool, commandStateMachine) { + var command commandStateMachine + cancellationID, isExternal := h.scheduledEventIDToCancellationID[initiatedeventID] + if !isExternal { + command = h.getCommand(makeCommandID(commandTypeChildWorkflow, workflowID)) + asChildWfCmd := command.(*childWorkflowCommandStateMachine) + asChildWfCmd.handleExternalWorkflowExecutionCancelRequested() + } else { + // this is cancellation for external workflow + command = h.getCommand(makeCommandID(commandTypeCancellation, cancellationID)) + command.handleCompletionEvent() + } + return isExternal, command +} + +func (h *commandsHelper) handleRequestCancelExternalWorkflowExecutionFailed(initiatedeventID int64, workflowID string) (bool, commandStateMachine) { + var command commandStateMachine + cancellationID, isExternal := h.scheduledEventIDToCancellationID[initiatedeventID] + if !isExternal { + // this is cancellation for child workflow only + command = h.getCommand(makeCommandID(commandTypeChildWorkflow, workflowID)) + command.handleCancelFailedEvent() + } else { + // this is cancellation for external workflow + command = h.getCommand(makeCommandID(commandTypeCancellation, cancellationID)) + command.handleCompletionEvent() + } + return isExternal, command +} + +func (h *commandsHelper) signalExternalWorkflowExecution( + namespace string, + workflowID string, + runID string, + signalName string, + input *commonpb.Payloads, + header *commonpb.Header, + signalID string, + childWorkflowOnly bool, +) commandStateMachine { + attributes := &commandpb.SignalExternalWorkflowExecutionCommandAttributes{ + Namespace: namespace, + Execution: &commonpb.WorkflowExecution{ + WorkflowId: workflowID, + RunId: runID, + }, + SignalName: signalName, + Input: input, + Control: signalID, + ChildWorkflowOnly: childWorkflowOnly, + Header: header, + } + command := h.newSignalExternalWorkflowStateMachine(attributes, signalID) + h.addCommand(command) + return command +} + +func (h *commandsHelper) addProtocolMessage(msgID string) commandStateMachine { + cmd := createNewCommand(enumspb.COMMAND_TYPE_PROTOCOL_MESSAGE) + cmd.Attributes = &commandpb.Command_ProtocolMessageCommandAttributes{ + ProtocolMessageCommandAttributes: &commandpb.ProtocolMessageCommandAttributes{MessageId: msgID}, + } + sm := &completeOnSendStateMachine{ + naiveCommandStateMachine: h.newNaiveCommandStateMachine(commandTypeProtocolMessage, msgID, cmd), + } + h.addCommand(sm) + return sm +} + +func (h *commandsHelper) upsertSearchAttributes(upsertID string, searchAttr *commonpb.SearchAttributes) commandStateMachine { + attributes := &commandpb.UpsertWorkflowSearchAttributesCommandAttributes{ + SearchAttributes: searchAttr, + } + command := h.newUpsertSearchAttributesStateMachine(attributes, upsertID) + h.addCommand(command) + return command +} + +func (h *commandsHelper) modifyProperties(changeID string, memo *commonpb.Memo) commandStateMachine { + attributes := &commandpb.ModifyWorkflowPropertiesCommandAttributes{ + UpsertedMemo: memo, + } + command := h.newModifyPropertiesStateMachine(attributes, changeID) + h.addCommand(command) + return command +} + +func (h *commandsHelper) handleSignalExternalWorkflowExecutionInitiated(initiatedEventID int64, signalID string) { + h.scheduledEventIDToSignalID[initiatedEventID] = signalID + command := h.getCommand(makeCommandID(commandTypeSignal, signalID)) + command.handleInitiatedEvent() +} + +func (h *commandsHelper) handleSignalExternalWorkflowExecutionCompleted(initiatedEventID int64) commandStateMachine { + command := h.getCommand(makeCommandID(commandTypeSignal, h.getSignalID(initiatedEventID))) + command.handleCompletionEvent() + return command +} + +func (h *commandsHelper) handleSignalExternalWorkflowExecutionFailed(initiatedEventID int64) commandStateMachine { + command := h.getCommand(makeCommandID(commandTypeSignal, h.getSignalID(initiatedEventID))) + command.handleCompletionEvent() + return command +} + +func (h *commandsHelper) getSignalID(initiatedEventID int64) string { + signalID, ok := h.scheduledEventIDToSignalID[initiatedEventID] + if !ok { + panic(fmt.Sprintf("unable to find signalID for initiatedEventID: %v", initiatedEventID)) + } + return signalID +} + +func (h *commandsHelper) startTimer( + attributes *commandpb.StartTimerCommandAttributes, + options TimerOptions, + dc converter.DataConverter, +) commandStateMachine { + startMetadata, err := buildUserMetadata(options.Summary, "", dc) + if err != nil { + panic(err) + } + command := h.newTimerCommandStateMachine(attributes, startMetadata) + h.addCommand(command) + return command +} + +func (h *commandsHelper) cancelTimer(timerID TimerID) commandStateMachine { + command := h.getCommand(makeCommandID(commandTypeTimer, timerID.id)) + command.cancel() + + return command +} + +func (h *commandsHelper) handleTimerClosed(timerID string) commandStateMachine { + command := h.getCommand(makeCommandID(commandTypeTimer, timerID)) + // If, for whatever reason, we were going to send a timer cancel command, don't do that anymore + // since we already know the timer is resolved. + possibleCancelID := makeCommandID(commandTypeCancelTimer, timerID) + h.removeCancelOfResolvedCommand(possibleCancelID) + command.handleCompletionEvent() + return command +} + +func (h *commandsHelper) handleTimerStarted(timerID string) { + command := h.getCommand(makeCommandID(commandTypeTimer, timerID)) + command.handleInitiatedEvent() +} + +func (h *commandsHelper) handleTimerCanceled(timerID string) { + command := h.getCommand(makeCommandID(commandTypeTimer, timerID)) + command.handleCanceledEvent() +} + +func (h *commandsHelper) handleChildWorkflowExecutionStarted(workflowID string) commandStateMachine { + command := h.getCommand(makeCommandID(commandTypeChildWorkflow, workflowID)) + command.handleStartedEvent() + return command +} + +func (h *commandsHelper) handleChildWorkflowExecutionClosed(workflowID string) commandStateMachine { + command := h.getCommand(makeCommandID(commandTypeChildWorkflow, workflowID)) + command.handleCompletionEvent() + return command +} + +func (h *commandsHelper) handleChildWorkflowExecutionCanceled(workflowID string) commandStateMachine { + command := h.getCommand(makeCommandID(commandTypeChildWorkflow, workflowID)) + command.handleCanceledEvent() + return command +} + +func (h *commandsHelper) getCommands(markAsSent bool) []*commandpb.Command { + var result []*commandpb.Command + for curr := h.orderedCommands.Front(); curr != nil; { + next := curr.Next() // get next item here as we might need to remove curr in the loop + d := curr.Value.(commandStateMachine) + command := d.getCommand() + if command != nil { + result = append(result, command) + } + + if markAsSent { + d.handleCommandSent() + } + + // remove completed command state machines + if d.getState() == commandStateCompleted { + h.orderedCommands.Remove(curr) + delete(h.commands, d.getID()) + } + + curr = next + } + + return result +} + +func (h *commandsHelper) isCancelExternalWorkflowEventForChildWorkflow(cancellationID string) bool { + // the cancellationID, i.e. Control in RequestCancelExternalWorkflowExecutionInitiatedEventAttributes + // will be empty if the event is for child workflow. + // for cancellation external workflow, Control in RequestCancelExternalWorkflowExecutionInitiatedEventAttributes + // will have a client generated sequence ID + return len(cancellationID) == 0 +} + +func (e *childWorkflowExistsWithId) Error() string { + return fmt.Sprintf("child workflow already exists with id: %v", e.id) +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_eager.go b/vendor/go.temporal.io/sdk/internal/internal_eager.go new file mode 100644 index 00000000000..e475f1d5f4e --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_eager.go @@ -0,0 +1,35 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +// eagerWorker is the minimal worker interface needed for eager activities and workflows +type eagerWorker interface { + // tryReserveSlot tries to reserver a task slot on the worker without blocking + // caller is expected to release the slot with releaseSlot + tryReserveSlot() *SlotPermit + // releaseSlot release a task slot acquired by tryReserveSlot + releaseSlot(permit *SlotPermit, reason SlotReleaseReason) + // pushEagerTask pushes a new eager workflow task to the workers task queue. + // should only be called with a reserved slot. + pushEagerTask(task eagerTask) +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_eager_activity.go b/vendor/go.temporal.io/sdk/internal/internal_eager_activity.go new file mode 100644 index 00000000000..c9bf5f4a4d8 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_eager_activity.go @@ -0,0 +1,149 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "fmt" + "sync" + + "go.temporal.io/api/workflowservice/v1" +) + +// eagerActivityExecutor is a worker-scoped executor for eager activities that +// are returned from workflow task completion responses. +type eagerActivityExecutor struct { + eagerActivityExecutorOptions + + activityWorker eagerWorker + heldSlotCount int + countLock sync.Mutex +} + +type eagerActivityExecutorOptions struct { + disabled bool + taskQueue string + // If 0, there is no maximum + maxConcurrent int +} + +// newEagerActivityExecutor creates a new worker-scoped executor without an +// activityWorker set. The activityWorker must be set on the responding executor +// before it will be able to execute activities. +func newEagerActivityExecutor(options eagerActivityExecutorOptions) *eagerActivityExecutor { + return &eagerActivityExecutor{eagerActivityExecutorOptions: options} +} + +func (e *eagerActivityExecutor) applyToRequest( + req *workflowservice.RespondWorkflowTaskCompletedRequest, +) []*SlotPermit { + // Don't allow more than this hardcoded amount per workflow task for now + const maxPerTask = 3 + reservedPermits := make([]*SlotPermit, 0) + + // Go over every command checking for activities that can be eagerly executed + eagerRequestsThisTask := 0 + for _, command := range req.Commands { + if attrs := command.GetScheduleActivityTaskCommandAttributes(); attrs != nil { + // If not present, disabled, not requested, no activity worker, on a + // different task queue, or reached max for task, we must mark as + // explicitly disabled + eagerDisallowed := e == nil || + e.disabled || + !attrs.RequestEagerExecution || + e.activityWorker == nil || + e.taskQueue != attrs.TaskQueue.GetName() || + eagerRequestsThisTask >= maxPerTask + if eagerDisallowed { + attrs.RequestEagerExecution = false + } else { + // If it has been requested, attempt to reserve one pending + maybePermit := e.reserveOnePendingSlot() + if maybePermit != nil { + reservedPermits = append(reservedPermits, maybePermit) + attrs.RequestEagerExecution = true + eagerRequestsThisTask++ + } else { + attrs.RequestEagerExecution = false + } + } + } + } + return reservedPermits +} + +func (e *eagerActivityExecutor) reserveOnePendingSlot() *SlotPermit { + // Confirm that, if we have a max, issued count isn't already there + e.countLock.Lock() + defer e.countLock.Unlock() + // Confirm that, if we have a max, held count isn't already there + if e.maxConcurrent > 0 && e.heldSlotCount >= e.maxConcurrent { + // No more room + return nil + } + // Reserve a spot for our request via a non-blocking attempt + maybePermit := e.activityWorker.tryReserveSlot() + if maybePermit != nil { + // Ensure that on release we decrement the held count + maybePermit.extraReleaseCallback = func() { + e.countLock.Lock() + defer e.countLock.Unlock() + e.heldSlotCount-- + } + e.heldSlotCount++ + } + return maybePermit +} + +func (e *eagerActivityExecutor) handleResponse( + resp *workflowservice.RespondWorkflowTaskCompletedResponse, + reservedPermits []*SlotPermit, +) { + // Ignore disabled or none present + amountSlotsReserved := len(reservedPermits) + if e == nil || e.activityWorker == nil || e.disabled || + (len(resp.GetActivityTasks()) == 0 && amountSlotsReserved == 0) { + return + } else if len(resp.GetActivityTasks()) > amountSlotsReserved { + panic(fmt.Sprintf("Unexpectedly received %v eager activities though we only requested %v", + len(resp.GetActivityTasks()), amountSlotsReserved)) + } + + // Give back unfulfilled slots and record for later use + unfulfilledSlots := amountSlotsReserved - len(resp.GetActivityTasks()) + // Release unneeded permits + for i := 0; i < unfulfilledSlots; i++ { + unneededPermit := reservedPermits[len(reservedPermits)-1] + reservedPermits = reservedPermits[:len(reservedPermits)-1] + e.activityWorker.releaseSlot(unneededPermit, SlotReleaseReasonUnused) + } + + // Start each activity asynchronously + for i, activity := range resp.GetActivityTasks() { + // Asynchronously execute + e.activityWorker.pushEagerTask( + eagerTask{ + task: &activityTask{task: activity, permit: reservedPermits[i]}, + permit: reservedPermits[i], + }) + } +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_eager_workflow.go b/vendor/go.temporal.io/sdk/internal/internal_eager_workflow.go new file mode 100644 index 00000000000..9b8eac0290c --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_eager_workflow.go @@ -0,0 +1,99 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "math/rand" + "sync" + "sync/atomic" + + "go.temporal.io/api/workflowservice/v1" +) + +// eagerWorkflowDispatcher is responsible for finding an available worker for an eager workflow task. +type eagerWorkflowDispatcher struct { + lock sync.RWMutex + workersByTaskQueue map[string][]eagerWorker +} + +// registerWorker registers a worker that can be used for eager workflow dispatch +func (e *eagerWorkflowDispatcher) registerWorker(worker *workflowWorker) { + e.lock.Lock() + defer e.lock.Unlock() + e.workersByTaskQueue[worker.executionParameters.TaskQueue] = append(e.workersByTaskQueue[worker.executionParameters.TaskQueue], worker.worker) +} + +// applyToRequest updates request if eager workflow dispatch is possible and returns the eagerWorkflowExecutor to use +func (e *eagerWorkflowDispatcher) applyToRequest(request *workflowservice.StartWorkflowExecutionRequest) *eagerWorkflowExecutor { + // Try every worker that is assigned to the desired task queue. + e.lock.RLock() + workers := e.workersByTaskQueue[request.GetTaskQueue().Name] + randWorkers := make([]eagerWorker, len(workers)) + // Copy the slice so we can release the lock. + copy(randWorkers, workers) + e.lock.RUnlock() + rand.Shuffle(len(randWorkers), func(i, j int) { randWorkers[i], randWorkers[j] = randWorkers[j], randWorkers[i] }) + for _, worker := range randWorkers { + maybePermit := worker.tryReserveSlot() + if maybePermit != nil { + request.RequestEagerExecution = true + return &eagerWorkflowExecutor{ + worker: worker, + permit: maybePermit, + } + } + } + return nil +} + +// eagerWorkflowExecutor is a worker-scoped executor for an eager workflow task. +type eagerWorkflowExecutor struct { + handledResponse atomic.Bool + worker eagerWorker + permit *SlotPermit +} + +// handleResponse of an eager workflow task from a StartWorkflowExecution request. +func (e *eagerWorkflowExecutor) handleResponse(response *workflowservice.PollWorkflowTaskQueueResponse) { + if !e.handledResponse.CompareAndSwap(false, true) { + panic("eagerWorkflowExecutor trying to handle multiple responses") + } + // Asynchronously execute the task + e.worker.pushEagerTask( + eagerTask{ + task: &eagerWorkflowTask{ + task: response, + }, + permit: e.permit, + }) +} + +// releaseUnused should be called if the executor cannot be used because no eager task was received. +// It will error if handleResponse was already called, as this would indicate misuse. +func (e *eagerWorkflowExecutor) releaseUnused() { + if e.handledResponse.CompareAndSwap(false, true) { + e.worker.releaseSlot(e.permit, SlotReleaseReasonUnused) + } else { + panic("trying to release an eagerWorkflowExecutor that was used") + } +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_event_handlers.go b/vendor/go.temporal.io/sdk/internal/internal_event_handlers.go new file mode 100644 index 00000000000..b7897df1ada --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_event_handlers.go @@ -0,0 +1,2078 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +// All code in this file is private to the package. + +import ( + "errors" + "fmt" + "reflect" + "sync" + "time" + + commandpb "go.temporal.io/api/command/v1" + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + failurepb "go.temporal.io/api/failure/v1" + historypb "go.temporal.io/api/history/v1" + protocolpb "go.temporal.io/api/protocol/v1" + taskqueuepb "go.temporal.io/api/taskqueue/v1" + "go.temporal.io/api/workflowservice/v1" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/durationpb" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" + ilog "go.temporal.io/sdk/internal/log" + "go.temporal.io/sdk/internal/protocol" + "go.temporal.io/sdk/log" +) + +const ( + queryResultSizeLimit = 2000000 // 2MB + changeVersionSearchAttrSizeLimit = 2048 +) + +// Assert that structs do indeed implement the interfaces +var ( + _ WorkflowEnvironment = (*workflowEnvironmentImpl)(nil) + _ workflowExecutionEventHandler = (*workflowExecutionEventHandlerImpl)(nil) +) + +type ( + // completionHandler Handler to indicate completion result + completionHandler func(result *commonpb.Payloads, err error) + + // workflowExecutionEventHandlerImpl handler to handle workflowExecutionEventHandler + workflowExecutionEventHandlerImpl struct { + *workflowEnvironmentImpl + workflowDefinition WorkflowDefinition + } + + scheduledTimer struct { + callback ResultHandler + handled bool + } + + scheduledActivity struct { + callback ResultHandler + waitForCancelRequest bool + handled bool + activityType ActivityType + } + + scheduledNexusOperation struct { + startedCallback func(operationID string, err error) + completedCallback func(result *commonpb.Payload, err error) + endpoint string + service string + operation string + } + + scheduledChildWorkflow struct { + resultCallback ResultHandler + startedCallback func(r WorkflowExecution, e error) + waitForCancellation bool + handled bool + } + + scheduledCancellation struct { + callback ResultHandler + handled bool + } + + scheduledSignal struct { + callback ResultHandler + handled bool + } + + sendCfg struct { + addCmd bool + pred func(*historypb.HistoryEvent) bool + } + + msgSendOpt func(so *sendCfg) + + outboxEntry struct { + eventPredicate func(*historypb.HistoryEvent) bool + msg *protocolpb.Message + } + + // workflowEnvironmentImpl an implementation of WorkflowEnvironment represents a environment for workflow execution. + workflowEnvironmentImpl struct { + workflowInfo *WorkflowInfo + + commandsHelper *commandsHelper + outbox []outboxEntry + sideEffectResult map[int64]*commonpb.Payloads + changeVersions map[string]Version + pendingLaTasks map[string]*localActivityTask + completedLaAttemptsThisWFT uint32 + // mutableSideEffect is a map for each mutable side effect ID where each key is the + // number of times the mutable side effect was called in a workflow + // execution per ID. + mutableSideEffect map[string]map[int]*commonpb.Payloads + unstartedLaTasks map[string]struct{} + openSessions map[string]*SessionInfo + + // Set of mutable side effect IDs that are recorded on the next task for use + // during replay to determine whether a command should be created. The keys + // are the user-provided IDs + "_" + the command counter. + mutableSideEffectsRecorded map[string]bool + // Records the number of times a mutable side effect was called per ID over the + // life of the workflow. Used to help distinguish multiple calls to MutableSideEffect in the same + // WorkflowTask. + mutableSideEffectCallCounter map[string]int + + // LocalActivities have a separate, individual counter instead of relying on actual commandEventIDs. + // This is because command IDs are only incremented on activity completion, which breaks + // local activities that are spawned in parallel as they would all share the same command ID + localActivityCounterID int64 + + sideEffectCounterID int64 + + currentReplayTime time.Time // Indicates current replay time of the command. + currentLocalTime time.Time // Local time when currentReplayTime was updated. + + completeHandler completionHandler // events completion handler + cancelHandler func() // A cancel handler to be invoked on a cancel notification + signalHandler func(name string, input *commonpb.Payloads, header *commonpb.Header) error // A signal handler to be invoked on a signal event + queryHandler func(queryType string, queryArgs *commonpb.Payloads, header *commonpb.Header) (*commonpb.Payloads, error) + updateHandler func(name string, id string, args *commonpb.Payloads, header *commonpb.Header, callbacks UpdateCallbacks) + + logger log.Logger + isReplay bool // flag to indicate if workflow is in replay mode + enableLoggingInReplay bool // flag to indicate if workflow should enable logging in replay mode + + metricsHandler metrics.Handler + registry *registry + dataConverter converter.DataConverter + failureConverter converter.FailureConverter + contextPropagators []ContextPropagator + deadlockDetectionTimeout time.Duration + sdkFlags *sdkFlags + sdkVersionUpdated bool + sdkVersion string + sdkNameUpdated bool + sdkName string + // Any update requests received in a workflow task before we have registered + // any handlers are not scheduled and are queued here until either their + // handler is registered or the event loop runs out of work and they are rejected. + bufferedUpdateRequests map[string][]func() + + protocols *protocol.Registry + } + + localActivityTask struct { + sync.Mutex + workflowTask *workflowTask + activityID string + params *ExecuteLocalActivityParams + callback LocalActivityResultHandler + wc *workflowExecutionContextImpl + canceled bool + cancelFunc func() + attempt int32 // attempt starting from 1 + attemptsThisWFT uint32 // Number of attempts started during this workflow task + pastFirstWFT bool // Set true once this LA has lived for more than one workflow task + retryPolicy *RetryPolicy + expireTime time.Time + scheduledTime time.Time // Time the activity was scheduled initially. + header *commonpb.Header + } + + localActivityMarkerData struct { + ActivityID string + ActivityType string + ReplayTime time.Time + Attempt int32 // record attempt, starting from 1. + Backoff time.Duration // retry backoff duration. + } +) + +var ( + // ErrUnknownMarkerName is returned if there is unknown marker name in the history. + ErrUnknownMarkerName = errors.New("unknown marker name") + // ErrMissingMarkerDetails is returned when marker details are nil. + ErrMissingMarkerDetails = errors.New("marker details are nil") + // ErrMissingMarkerDataKey is returned when marker details doesn't have data key. + ErrMissingMarkerDataKey = errors.New("marker key is missing in details") + // ErrUnknownHistoryEvent is returned if there is an unknown event in history and the SDK needs to handle it + ErrUnknownHistoryEvent = errors.New("unknown history event") +) + +func newWorkflowExecutionEventHandler( + workflowInfo *WorkflowInfo, + completeHandler completionHandler, + logger log.Logger, + enableLoggingInReplay bool, + metricsHandler metrics.Handler, + registry *registry, + dataConverter converter.DataConverter, + failureConverter converter.FailureConverter, + contextPropagators []ContextPropagator, + deadlockDetectionTimeout time.Duration, + capabilities *workflowservice.GetSystemInfoResponse_Capabilities, +) workflowExecutionEventHandler { + context := &workflowEnvironmentImpl{ + workflowInfo: workflowInfo, + commandsHelper: newCommandsHelper(), + sideEffectResult: make(map[int64]*commonpb.Payloads), + mutableSideEffect: make(map[string]map[int]*commonpb.Payloads), + changeVersions: make(map[string]Version), + pendingLaTasks: make(map[string]*localActivityTask), + unstartedLaTasks: make(map[string]struct{}), + openSessions: make(map[string]*SessionInfo), + completeHandler: completeHandler, + enableLoggingInReplay: enableLoggingInReplay, + registry: registry, + dataConverter: dataConverter, + failureConverter: failureConverter, + contextPropagators: contextPropagators, + deadlockDetectionTimeout: deadlockDetectionTimeout, + protocols: protocol.NewRegistry(), + mutableSideEffectCallCounter: make(map[string]int), + sdkFlags: newSDKFlags(capabilities), + bufferedUpdateRequests: make(map[string][]func()), + } + // Attempt to skip 1 log level to remove the ReplayLogger from the stack. + context.logger = log.Skip(ilog.NewReplayLogger( + log.With(logger, + tagWorkflowType, workflowInfo.WorkflowType.Name, + tagWorkflowID, workflowInfo.WorkflowExecution.ID, + tagRunID, workflowInfo.WorkflowExecution.RunID, + tagAttempt, workflowInfo.Attempt, + ), + &context.isReplay, + &context.enableLoggingInReplay), 1) + + if metricsHandler != nil { + context.metricsHandler = metrics.NewReplayAwareHandler(&context.isReplay, metricsHandler). + WithTags(metrics.WorkflowTags(workflowInfo.WorkflowType.Name)) + } + + return &workflowExecutionEventHandlerImpl{context, nil} +} + +func (s *scheduledTimer) handle(result *commonpb.Payloads, err error) { + if s.handled { + panic(fmt.Sprintf("timer already handled %v", s)) + } + s.handled = true + s.callback(result, err) +} + +func (s *scheduledActivity) handle(result *commonpb.Payloads, err error) { + if s.handled { + panic(fmt.Sprintf("activity already handled %v", s)) + } + s.handled = true + s.callback(result, err) +} + +func (s *scheduledChildWorkflow) handle(result *commonpb.Payloads, err error) { + if s.handled { + panic(fmt.Sprintf("child workflow already handled %v", s)) + } + s.handled = true + s.resultCallback(result, err) +} + +func (s *scheduledChildWorkflow) handleFailedToStart(result *commonpb.Payloads, err error) { + if s.handled { + panic(fmt.Sprintf("child workflow already handled %v", s)) + } + s.handled = true + s.resultCallback(result, err) + s.startedCallback(WorkflowExecution{}, err) +} + +func (t *localActivityTask) cancel() { + t.Lock() + t.canceled = true + if t.cancelFunc != nil { + t.cancelFunc() + } + t.Unlock() +} + +func (s *scheduledCancellation) handle(result *commonpb.Payloads, err error) { + if s.handled { + panic(fmt.Sprintf("cancellation already handled %v", s)) + } + s.handled = true + s.callback(result, err) +} + +func (s *scheduledSignal) handle(result *commonpb.Payloads, err error) { + if s.handled { + panic(fmt.Sprintf("signal already handled %v", s)) + } + s.handled = true + s.callback(result, err) +} + +func (wc *workflowEnvironmentImpl) takeOutgoingMessages() []*protocolpb.Message { + retval := make([]*protocolpb.Message, 0, len(wc.outbox)) + for _, entry := range wc.outbox { + retval = append(retval, entry.msg) + } + wc.outbox = nil + return retval +} + +func (wc *workflowEnvironmentImpl) ScheduleUpdate(name string, id string, args *commonpb.Payloads, hdr *commonpb.Header, callbacks UpdateCallbacks) { + wc.updateHandler(name, id, args, hdr, callbacks) +} + +func withExpectedEventPredicate(pred func(*historypb.HistoryEvent) bool) msgSendOpt { + return func(so *sendCfg) { + so.addCmd = true + so.pred = pred + } +} + +func (wc *workflowEnvironmentImpl) Send(msg *protocolpb.Message, opts ...msgSendOpt) { + sendCfg := sendCfg{ + pred: func(*historypb.HistoryEvent) bool { return false }, + } + for _, opt := range opts { + opt(&sendCfg) + } + canSendCmd := wc.sdkFlags.tryUse(SDKFlagProtocolMessageCommand, !wc.isReplay) + if canSendCmd && sendCfg.addCmd { + wc.commandsHelper.addProtocolMessage(msg.Id) + } + wc.outbox = append(wc.outbox, outboxEntry{msg: msg, eventPredicate: sendCfg.pred}) +} + +func (wc *workflowEnvironmentImpl) getNewSdkNameAndReset() string { + if wc.sdkNameUpdated { + wc.sdkNameUpdated = false + return wc.sdkName + } + return "" +} + +func (wc *workflowEnvironmentImpl) getNewSdkVersionAndReset() string { + if wc.sdkVersionUpdated { + wc.sdkVersionUpdated = false + return wc.sdkVersion + } + return "" +} + +func (wc *workflowEnvironmentImpl) getNextLocalActivityID() string { + wc.localActivityCounterID++ + return getStringID(wc.localActivityCounterID) +} + +func (wc *workflowEnvironmentImpl) getNextSideEffectID() int64 { + wc.sideEffectCounterID++ + return wc.sideEffectCounterID +} + +func (wc *workflowEnvironmentImpl) WorkflowInfo() *WorkflowInfo { + return wc.workflowInfo +} + +func (wc *workflowEnvironmentImpl) TypedSearchAttributes() SearchAttributes { + return convertToTypedSearchAttributes(wc.logger, wc.workflowInfo.SearchAttributes.GetIndexedFields()) +} + +func (wc *workflowEnvironmentImpl) Complete(result *commonpb.Payloads, err error) { + wc.completeHandler(result, err) +} + +func (wc *workflowEnvironmentImpl) RequestCancelChildWorkflow(namespace string, workflowID string) { + // For cancellation of child workflow only, we do not use cancellation ID and run ID + wc.commandsHelper.requestCancelExternalWorkflowExecution(namespace, workflowID, "", "", true) +} + +func (wc *workflowEnvironmentImpl) RequestCancelExternalWorkflow(namespace, workflowID, runID string, callback ResultHandler) { + // for cancellation of external workflow, we have to use cancellation ID and set isChildWorkflowOnly to false + cancellationID := wc.GenerateSequenceID() + command := wc.commandsHelper.requestCancelExternalWorkflowExecution(namespace, workflowID, runID, cancellationID, false) + command.setData(&scheduledCancellation{callback: callback}) +} + +func (wc *workflowEnvironmentImpl) SignalExternalWorkflow( + namespace string, + workflowID string, + runID string, + signalName string, + input *commonpb.Payloads, + _ /* THIS IS FOR TEST FRAMEWORK. DO NOT USE HERE. */ interface{}, + header *commonpb.Header, + childWorkflowOnly bool, + callback ResultHandler, +) { + signalID := wc.GenerateSequenceID() + command := wc.commandsHelper.signalExternalWorkflowExecution(namespace, workflowID, runID, signalName, input, + header, signalID, childWorkflowOnly) + command.setData(&scheduledSignal{callback: callback}) +} + +func (wc *workflowEnvironmentImpl) UpsertSearchAttributes(attributes map[string]interface{}) error { + // This has to be used in WorkflowEnvironment implementations instead of in Workflow for testsuite mock purpose. + attr, err := validateAndSerializeSearchAttributes(attributes) + if err != nil { + return err + } + + var upsertID string + if changeVersion, ok := attributes[TemporalChangeVersion]; ok { + // to ensure backward compatibility on searchable GetVersion, use latest changeVersion as upsertID + upsertID = changeVersion.([]string)[0] + } else { + upsertID = wc.GenerateSequenceID() + } + + wc.commandsHelper.upsertSearchAttributes(upsertID, attr) + wc.updateWorkflowInfoWithSearchAttributes(attr) // this is for getInfo correctness + return nil +} + +func (wc *workflowEnvironmentImpl) UpsertTypedSearchAttributes(attributes SearchAttributes) error { + rawSearchAttributes, err := serializeTypedSearchAttributes(attributes.untypedValue) + if err != nil { + return err + } + + if _, ok := rawSearchAttributes.GetIndexedFields()[TemporalChangeVersion]; ok { + return errors.New("TemporalChangeVersion is a reserved key that cannot be set, please use other key") + } + + attr := make(map[string]interface{}) + for k, v := range rawSearchAttributes.GetIndexedFields() { + attr[k] = v + } + return wc.UpsertSearchAttributes(attr) +} + +func (wc *workflowEnvironmentImpl) updateWorkflowInfoWithSearchAttributes(attributes *commonpb.SearchAttributes) { + wc.workflowInfo.SearchAttributes = mergeSearchAttributes(wc.workflowInfo.SearchAttributes, attributes) +} + +func mergeSearchAttributes(current, upsert *commonpb.SearchAttributes) *commonpb.SearchAttributes { + if current == nil || len(current.IndexedFields) == 0 { + if upsert == nil || len(upsert.IndexedFields) == 0 { + return nil + } + current = &commonpb.SearchAttributes{ + IndexedFields: make(map[string]*commonpb.Payload), + } + } + + fields := current.IndexedFields + for k, v := range upsert.IndexedFields { + fields[k] = v + } + return current +} + +func validateAndSerializeSearchAttributes(attributes map[string]interface{}) (*commonpb.SearchAttributes, error) { + if len(attributes) == 0 { + return nil, errSearchAttributesNotSet + } + attr, err := serializeUntypedSearchAttributes(attributes) + if err != nil { + return nil, err + } + return attr, nil +} + +func (wc *workflowEnvironmentImpl) UpsertMemo(memoMap map[string]interface{}) error { + // This has to be used in WorkflowEnvironment implementations instead of in Workflow for testsuite mock purpose. + memo, err := validateAndSerializeMemo(memoMap, wc.dataConverter) + if err != nil { + return err + } + + changeID := wc.GenerateSequenceID() + wc.commandsHelper.modifyProperties(changeID, memo) + wc.updateWorkflowInfoWithMemo(memo) // this is for getInfo correctness + return nil +} + +func (wc *workflowEnvironmentImpl) updateWorkflowInfoWithMemo(memo *commonpb.Memo) { + wc.workflowInfo.Memo = mergeMemo(wc.workflowInfo.Memo, memo) +} + +func mergeMemo(current, upsert *commonpb.Memo) *commonpb.Memo { + if current == nil || len(current.Fields) == 0 { + if upsert == nil || len(upsert.Fields) == 0 { + return nil + } + current = &commonpb.Memo{ + Fields: make(map[string]*commonpb.Payload), + } + } + + fields := current.Fields + for k, v := range upsert.Fields { + if v.Data == nil { + delete(fields, k) + } else { + fields[k] = v + } + } + return current +} + +func validateAndSerializeMemo(memoMap map[string]interface{}, dc converter.DataConverter) (*commonpb.Memo, error) { + if len(memoMap) == 0 { + return nil, errMemoNotSet + } + return getWorkflowMemo(memoMap, dc) +} + +func (wc *workflowEnvironmentImpl) RegisterCancelHandler(handler func()) { + wrappedHandler := func() { + handler() + } + wc.cancelHandler = wrappedHandler +} + +func (wc *workflowEnvironmentImpl) ExecuteChildWorkflow( + params ExecuteWorkflowParams, callback ResultHandler, startedHandler func(r WorkflowExecution, e error), +) { + if params.WorkflowID == "" { + params.WorkflowID = wc.workflowInfo.WorkflowExecution.RunID + "_" + wc.GenerateSequenceID() + } + memo, err := getWorkflowMemo(params.Memo, wc.dataConverter) + if err != nil { + if wc.sdkFlags.tryUse(SDKFlagChildWorkflowErrorExecution, !wc.isReplay) { + startedHandler(WorkflowExecution{}, &ChildWorkflowExecutionAlreadyStartedError{}) + } + callback(nil, err) + return + } + searchAttr, err := serializeSearchAttributes(params.SearchAttributes, params.TypedSearchAttributes) + if err != nil { + if wc.sdkFlags.tryUse(SDKFlagChildWorkflowErrorExecution, !wc.isReplay) { + startedHandler(WorkflowExecution{}, &ChildWorkflowExecutionAlreadyStartedError{}) + } + callback(nil, err) + return + } + + attributes := &commandpb.StartChildWorkflowExecutionCommandAttributes{} + + attributes.Namespace = params.Namespace + attributes.TaskQueue = &taskqueuepb.TaskQueue{Name: params.TaskQueueName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} + attributes.WorkflowId = params.WorkflowID + attributes.WorkflowExecutionTimeout = durationpb.New(params.WorkflowExecutionTimeout) + attributes.WorkflowRunTimeout = durationpb.New(params.WorkflowRunTimeout) + attributes.WorkflowTaskTimeout = durationpb.New(params.WorkflowTaskTimeout) + attributes.Input = params.Input + attributes.WorkflowType = &commonpb.WorkflowType{Name: params.WorkflowType.Name} + attributes.WorkflowIdReusePolicy = params.WorkflowIDReusePolicy + attributes.ParentClosePolicy = params.ParentClosePolicy + attributes.RetryPolicy = params.RetryPolicy + attributes.Header = params.Header + attributes.Memo = memo + attributes.SearchAttributes = searchAttr + if len(params.CronSchedule) > 0 { + attributes.CronSchedule = params.CronSchedule + } + attributes.InheritBuildId = determineInheritBuildIdFlagForCommand( + params.VersioningIntent, wc.workflowInfo.TaskQueueName, params.TaskQueueName) + + startMetadata, err := buildUserMetadata(params.staticSummary, params.staticDetails, wc.dataConverter) + if err != nil { + callback(nil, err) + return + } + + command, err := wc.commandsHelper.startChildWorkflowExecution(attributes, startMetadata) + if _, ok := err.(*childWorkflowExistsWithId); ok { + if wc.sdkFlags.tryUse(SDKFlagChildWorkflowErrorExecution, !wc.isReplay) { + startedHandler(WorkflowExecution{}, &ChildWorkflowExecutionAlreadyStartedError{}) + } + callback(nil, &ChildWorkflowExecutionAlreadyStartedError{}) + return + } + command.setData(&scheduledChildWorkflow{ + resultCallback: callback, + startedCallback: startedHandler, + waitForCancellation: params.WaitForCancellation, + }) + + wc.logger.Debug("ExecuteChildWorkflow", + tagChildWorkflowID, params.WorkflowID, + tagWorkflowType, params.WorkflowType.Name) +} + +func (wc *workflowEnvironmentImpl) ExecuteNexusOperation(params executeNexusOperationParams, callback func(*commonpb.Payload, error), startedHandler func(opID string, e error)) int64 { + seq := wc.GenerateSequence() + scheduleTaskAttr := &commandpb.ScheduleNexusOperationCommandAttributes{ + Endpoint: params.client.Endpoint(), + Service: params.client.Service(), + Operation: params.operation, + Input: params.input, + ScheduleToCloseTimeout: durationpb.New(params.options.ScheduleToCloseTimeout), + NexusHeader: params.nexusHeader, + } + + command := wc.commandsHelper.scheduleNexusOperation(seq, scheduleTaskAttr) + command.setData(&scheduledNexusOperation{ + startedCallback: startedHandler, + completedCallback: callback, + endpoint: params.client.Endpoint(), + service: params.client.Service(), + operation: params.operation, + }) + + wc.logger.Debug("ScheduleNexusOperation", + tagNexusEndpoint, params.client.Endpoint(), + tagNexusService, params.client.Service(), + tagNexusOperation, params.operation, + ) + + return command.seq +} + +func (wc *workflowEnvironmentImpl) RequestCancelNexusOperation(seq int64) { + command := wc.commandsHelper.requestCancelNexusOperation(seq) + data := command.getData().(*scheduledNexusOperation) + + // Make sure to unblock the futures. + if command.getState() == commandStateCreated || command.getState() == commandStateCommandSent { + if data.startedCallback != nil { + data.startedCallback("", ErrCanceled) + data.startedCallback = nil + } + if data.completedCallback != nil { + data.completedCallback(nil, ErrCanceled) + data.completedCallback = nil + } + } + wc.logger.Debug("RequestCancelNexusOperation", + tagNexusEndpoint, data.endpoint, + tagNexusService, data.service, + tagNexusOperation, data.operation, + ) +} + +func (wc *workflowEnvironmentImpl) RegisterSignalHandler( + handler func(name string, input *commonpb.Payloads, header *commonpb.Header) error, +) { + wc.signalHandler = handler +} + +func (wc *workflowEnvironmentImpl) RegisterQueryHandler( + handler func(string, *commonpb.Payloads, *commonpb.Header) (*commonpb.Payloads, error), +) { + wc.queryHandler = handler +} + +func (wc *workflowEnvironmentImpl) RegisterUpdateHandler( + handler func(string, string, *commonpb.Payloads, *commonpb.Header, UpdateCallbacks), +) { + wc.updateHandler = handler +} + +func (wc *workflowEnvironmentImpl) GetLogger() log.Logger { + return wc.logger +} + +func (wc *workflowEnvironmentImpl) GetMetricsHandler() metrics.Handler { + return wc.metricsHandler +} + +func (wc *workflowEnvironmentImpl) GetDataConverter() converter.DataConverter { + return wc.dataConverter +} + +func (wc *workflowEnvironmentImpl) GetFailureConverter() converter.FailureConverter { + return wc.failureConverter +} + +func (wc *workflowEnvironmentImpl) GetContextPropagators() []ContextPropagator { + return wc.contextPropagators +} + +func (wc *workflowEnvironmentImpl) IsReplaying() bool { + return wc.isReplay +} + +func (wc *workflowEnvironmentImpl) GenerateSequenceID() string { + return getStringID(wc.GenerateSequence()) +} + +func (wc *workflowEnvironmentImpl) GenerateSequence() int64 { + return wc.commandsHelper.getNextID() +} + +func (wc *workflowEnvironmentImpl) CreateNewCommand(commandType enumspb.CommandType) *commandpb.Command { + return &commandpb.Command{ + CommandType: commandType, + } +} + +func (wc *workflowEnvironmentImpl) ExecuteActivity(parameters ExecuteActivityParams, callback ResultHandler) ActivityID { + scheduleTaskAttr := &commandpb.ScheduleActivityTaskCommandAttributes{} + scheduleID := wc.GenerateSequence() + if parameters.ActivityID == "" { + scheduleTaskAttr.ActivityId = getStringID(scheduleID) + } else { + scheduleTaskAttr.ActivityId = parameters.ActivityID + } + activityID := scheduleTaskAttr.GetActivityId() + scheduleTaskAttr.ActivityType = &commonpb.ActivityType{Name: parameters.ActivityType.Name} + scheduleTaskAttr.TaskQueue = &taskqueuepb.TaskQueue{Name: parameters.TaskQueueName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} + scheduleTaskAttr.Input = parameters.Input + scheduleTaskAttr.ScheduleToCloseTimeout = durationpb.New(parameters.ScheduleToCloseTimeout) + scheduleTaskAttr.StartToCloseTimeout = durationpb.New(parameters.StartToCloseTimeout) + scheduleTaskAttr.ScheduleToStartTimeout = durationpb.New(parameters.ScheduleToStartTimeout) + scheduleTaskAttr.HeartbeatTimeout = durationpb.New(parameters.HeartbeatTimeout) + scheduleTaskAttr.RetryPolicy = parameters.RetryPolicy + scheduleTaskAttr.Header = parameters.Header + // We set this as true if not disabled on the params knowing it will be set as + // false just before request by the eager activity executor if eager activity + // execution is otherwise disallowed + scheduleTaskAttr.RequestEagerExecution = !parameters.DisableEagerExecution + scheduleTaskAttr.UseWorkflowBuildId = determineInheritBuildIdFlagForCommand( + parameters.VersioningIntent, wc.workflowInfo.TaskQueueName, parameters.TaskQueueName) + + command := wc.commandsHelper.scheduleActivityTask(scheduleID, scheduleTaskAttr) + command.setData(&scheduledActivity{ + callback: callback, + waitForCancelRequest: parameters.WaitForCancellation, + activityType: parameters.ActivityType, + }) + + wc.logger.Debug("ExecuteActivity", + tagActivityID, activityID, + tagActivityType, scheduleTaskAttr.ActivityType.GetName()) + + return ActivityID{id: activityID} +} + +func (wc *workflowEnvironmentImpl) RequestCancelActivity(activityID ActivityID) { + command := wc.commandsHelper.requestCancelActivityTask(activityID.id) + activity := command.getData().(*scheduledActivity) + if activity.handled { + return + } + + if command.isDone() || !activity.waitForCancelRequest { + activity.handle(nil, ErrCanceled) + } + + wc.logger.Debug("RequestCancelActivity", tagActivityID, activityID) +} + +func (wc *workflowEnvironmentImpl) ExecuteLocalActivity(params ExecuteLocalActivityParams, callback LocalActivityResultHandler) LocalActivityID { + activityID := wc.getNextLocalActivityID() + task := newLocalActivityTask(params, callback, activityID) + wc.pendingLaTasks[activityID] = task + wc.unstartedLaTasks[activityID] = struct{}{} + return LocalActivityID{id: activityID} +} + +func newLocalActivityTask(params ExecuteLocalActivityParams, callback LocalActivityResultHandler, activityID string) *localActivityTask { + task := &localActivityTask{ + activityID: activityID, + params: ¶ms, + callback: callback, + retryPolicy: params.RetryPolicy, + attempt: params.Attempt, + header: params.Header, + scheduledTime: time.Now(), + } + + if params.ScheduleToCloseTimeout > 0 { + task.expireTime = params.ScheduledTime.Add(params.ScheduleToCloseTimeout) + } + return task +} + +func (wc *workflowEnvironmentImpl) RequestCancelLocalActivity(activityID LocalActivityID) { + if task, ok := wc.pendingLaTasks[activityID.id]; ok { + task.cancel() + } +} + +func (wc *workflowEnvironmentImpl) SetCurrentReplayTime(replayTime time.Time) { + if replayTime.Before(wc.currentReplayTime) { + return + } + wc.currentReplayTime = replayTime + wc.currentLocalTime = time.Now() +} + +func (wc *workflowEnvironmentImpl) Now() time.Time { + return wc.currentReplayTime +} + +func (wc *workflowEnvironmentImpl) NewTimer(d time.Duration, options TimerOptions, callback ResultHandler) *TimerID { + if d < 0 { + callback(nil, fmt.Errorf("negative duration provided %v", d)) + return nil + } + if d == 0 { + callback(nil, nil) + return nil + } + + timerID := wc.GenerateSequenceID() + startTimerAttr := &commandpb.StartTimerCommandAttributes{} + startTimerAttr.TimerId = timerID + startTimerAttr.StartToFireTimeout = durationpb.New(d) + + command := wc.commandsHelper.startTimer(startTimerAttr, options, wc.GetDataConverter()) + command.setData(&scheduledTimer{callback: callback}) + + wc.logger.Debug("NewTimer", + tagTimerID, startTimerAttr.GetTimerId(), + "Duration", d) + + return &TimerID{id: timerID} +} + +func (wc *workflowEnvironmentImpl) RequestCancelTimer(timerID TimerID) { + command := wc.commandsHelper.cancelTimer(timerID) + timer := command.getData().(*scheduledTimer) + if timer != nil { + if timer.handled { + return + } + timer.handle(nil, ErrCanceled) + } + wc.logger.Debug("RequestCancelTimer", tagTimerID, timerID) +} + +func validateVersion(changeID string, version, minSupported, maxSupported Version) { + if version < minSupported { + panicIllegalState(fmt.Sprintf("[TMPRL1100] Workflow code removed support of version %v. "+ + "for \"%v\" changeID. The oldest supported version is %v", + version, changeID, minSupported)) + } + if version > maxSupported { + panicIllegalState(fmt.Sprintf("[TMPRL1100] Workflow code is too old to support version %v "+ + "for \"%v\" changeID. The maximum supported version is %v", + version, changeID, maxSupported)) + } +} + +func (wc *workflowEnvironmentImpl) GetVersion(changeID string, minSupported, maxSupported Version) Version { + if version, ok := wc.changeVersions[changeID]; ok { + validateVersion(changeID, version, minSupported, maxSupported) + return version + } + + var version Version + if wc.isReplay { + // GetVersion for changeID is called first time in replay mode, use DefaultVersion + version = DefaultVersion + } else { + // GetVersion for changeID is called first time (non-replay mode), generate a marker command for it. + // Also upsert search attributes to enable ability to search by changeVersion. + version = maxSupported + changeVersionSA := createSearchAttributesForChangeVersion(changeID, version, wc.changeVersions) + attr, err := validateAndSerializeSearchAttributes(changeVersionSA) + if err != nil { + wc.logger.Warn(fmt.Sprintf("Failed to seralize %s search attribute with: %v", TemporalChangeVersion, err)) + } else { + // Server has a limit for the max size of a single search attribute value. If we exceed the default limit + // do not try to upsert as it will cause the workflow to fail. + updateSearchAttribute := true + if wc.sdkFlags.tryUse(SDKFlagLimitChangeVersionSASize, !wc.isReplay) && len(attr.IndexedFields[TemporalChangeVersion].GetData()) >= changeVersionSearchAttrSizeLimit { + wc.logger.Warn(fmt.Sprintf("Serialized size of %s search attribute update would "+ + "exceed the maximum value size. Skipping this upsert. Be aware that your "+ + "visibility records will not include the following patch: %s", TemporalChangeVersion, getChangeVersion(changeID, version)), + ) + updateSearchAttribute = false + } + wc.commandsHelper.recordVersionMarker(changeID, version, wc.GetDataConverter(), updateSearchAttribute) + if updateSearchAttribute { + _ = wc.UpsertSearchAttributes(changeVersionSA) + } + } + } + + validateVersion(changeID, version, minSupported, maxSupported) + wc.changeVersions[changeID] = version + return version +} + +func createSearchAttributesForChangeVersion(changeID string, version Version, existingChangeVersions map[string]Version) map[string]interface{} { + return map[string]interface{}{ + TemporalChangeVersion: getChangeVersions(changeID, version, existingChangeVersions), + } +} + +func getChangeVersions(changeID string, version Version, existingChangeVersions map[string]Version) []string { + res := []string{getChangeVersion(changeID, version)} + for k, v := range existingChangeVersions { + res = append(res, getChangeVersion(k, v)) + } + return res +} + +func getChangeVersion(changeID string, version Version) string { + return fmt.Sprintf("%s-%v", changeID, version) +} + +func (wc *workflowEnvironmentImpl) SideEffect(f func() (*commonpb.Payloads, error), callback ResultHandler) { + sideEffectID := wc.getNextSideEffectID() + var result *commonpb.Payloads + if wc.isReplay { + var ok bool + result, ok = wc.sideEffectResult[sideEffectID] + if !ok { + keys := make([]int64, 0, len(wc.sideEffectResult)) + for k := range wc.sideEffectResult { + keys = append(keys, k) + } + panicIllegalState(fmt.Sprintf("[TMPRL1100] No cached result found for side effectID=%v. KnownSideEffects=%v", + sideEffectID, keys)) + } + + // Once the SideEffect has been consumed, we can free the referenced payload + // to reduce memory pressure + delete(wc.sideEffectResult, sideEffectID) + wc.logger.Debug("SideEffect returning already calculated result.", + tagSideEffectID, sideEffectID) + } else { + var err error + result, err = f() + if err != nil { + callback(result, err) + return + } + } + + wc.commandsHelper.recordSideEffectMarker(sideEffectID, result, wc.dataConverter) + + callback(result, nil) + wc.logger.Debug("SideEffect Marker added", tagSideEffectID, sideEffectID) +} + +func (wc *workflowEnvironmentImpl) TryUse(flag sdkFlag) bool { + return wc.sdkFlags.tryUse(flag, !wc.isReplay) +} + +func (wc *workflowEnvironmentImpl) QueueUpdate(name string, f func()) { + wc.bufferedUpdateRequests[name] = append(wc.bufferedUpdateRequests[name], f) +} + +func (wc *workflowEnvironmentImpl) HandleQueuedUpdates(name string) { + if bufferedUpdateRequests, ok := wc.bufferedUpdateRequests[name]; ok { + for _, request := range bufferedUpdateRequests { + request() + } + delete(wc.bufferedUpdateRequests, name) + } +} + +func (wc *workflowEnvironmentImpl) DrainUnhandledUpdates() bool { + anyExecuted := false + // Check if any buffered update requests remain when we have no more coroutines to run and let them schedule so they are rejected. + // Generally iterating a map in workflow code is bad because it is non deterministic + // this case is fine since all these update handles will be rejected and not recorded in history. + for name, requests := range wc.bufferedUpdateRequests { + for _, request := range requests { + request() + anyExecuted = true + } + delete(wc.bufferedUpdateRequests, name) + } + return anyExecuted +} + +// lookupMutableSideEffect gets the current value of the MutableSideEffect for id for the +// current call count of id. +func (wc *workflowEnvironmentImpl) lookupMutableSideEffect(id string) *commonpb.Payloads { + // Fail if ID not found + callCountPayloads := wc.mutableSideEffect[id] + if len(callCountPayloads) == 0 { + return nil + } + currentCallCount := wc.mutableSideEffectCallCounter[id] + + // Find the most recent call at/before the current call count + var payloads *commonpb.Payloads + payloadIndex := -1 + for callCount, maybePayloads := range callCountPayloads { + if callCount <= currentCallCount && callCount > payloadIndex { + payloads = maybePayloads + payloadIndex = callCount + } + } + + // Garbage collect old entries + for callCount := range callCountPayloads { + if callCount <= currentCallCount && callCount != payloadIndex { + delete(callCountPayloads, callCount) + } + } + + return payloads +} + +func (wc *workflowEnvironmentImpl) MutableSideEffect(id string, f func() interface{}, equals func(a, b interface{}) bool) converter.EncodedValue { + wc.mutableSideEffectCallCounter[id]++ + callCount := wc.mutableSideEffectCallCounter[id] + + if result := wc.lookupMutableSideEffect(id); result != nil { + encodedResult := newEncodedValue(result, wc.GetDataConverter()) + if wc.isReplay { + // During replay, we only generate a command if there was a known marker + // recorded on the next task. We have to append the current command + // counter to the user-provided ID to avoid duplicates. + if wc.mutableSideEffectsRecorded[fmt.Sprintf("%v_%v", id, wc.commandsHelper.getNextID())] { + return wc.recordMutableSideEffect(id, callCount, result) + } + return encodedResult + } + + newValue := f() + if wc.isEqualValue(newValue, result, equals) { + return encodedResult + } + + return wc.recordMutableSideEffect(id, callCount, wc.encodeValue(newValue)) + } + + if wc.isReplay { + // This should not happen + panicIllegalState(fmt.Sprintf("[TMPRL1100] Non deterministic workflow code change detected. MutableSideEffect API call doesn't have a correspondent event in the workflow history. MutableSideEffect ID: %s", id)) + } + + return wc.recordMutableSideEffect(id, callCount, wc.encodeValue(f())) +} + +func (wc *workflowEnvironmentImpl) isEqualValue(newValue interface{}, encodedOldValue *commonpb.Payloads, equals func(a, b interface{}) bool) bool { + if newValue == nil { + // new value is nil + newEncodedValue := wc.encodeValue(nil) + return proto.Equal(newEncodedValue, encodedOldValue) + } + + oldValue := decodeValue(newEncodedValue(encodedOldValue, wc.GetDataConverter()), newValue) + return equals(newValue, oldValue) +} + +func decodeValue(encodedValue converter.EncodedValue, value interface{}) interface{} { + // We need to decode oldValue out of encodedValue, first we need to prepare valuePtr as the same type as value + valuePtr := reflect.New(reflect.TypeOf(value)).Interface() + if err := encodedValue.Get(valuePtr); err != nil { + panic(err) + } + decodedValue := reflect.ValueOf(valuePtr).Elem().Interface() + return decodedValue +} + +func (wc *workflowEnvironmentImpl) encodeValue(value interface{}) *commonpb.Payloads { + payload, err := wc.encodeArg(value) + if err != nil { + panic(err) + } + return payload +} + +func (wc *workflowEnvironmentImpl) encodeArg(arg interface{}) (*commonpb.Payloads, error) { + return wc.GetDataConverter().ToPayloads(arg) +} + +func (wc *workflowEnvironmentImpl) recordMutableSideEffect(id string, callCountHint int, data *commonpb.Payloads) converter.EncodedValue { + details, err := encodeArgs(wc.GetDataConverter(), []interface{}{id, data}) + if err != nil { + panic(err) + } + wc.commandsHelper.recordMutableSideEffectMarker(id, callCountHint, details, wc.dataConverter) + if wc.mutableSideEffect[id] == nil { + wc.mutableSideEffect[id] = make(map[int]*commonpb.Payloads) + } + wc.mutableSideEffect[id][callCountHint] = data + return newEncodedValue(data, wc.GetDataConverter()) +} + +func (wc *workflowEnvironmentImpl) AddSession(sessionInfo *SessionInfo) { + wc.openSessions[sessionInfo.SessionID] = sessionInfo +} + +func (wc *workflowEnvironmentImpl) RemoveSession(sessionID string) { + delete(wc.openSessions, sessionID) +} + +func (wc *workflowEnvironmentImpl) getOpenSessions() []*SessionInfo { + openSessions := make([]*SessionInfo, 0, len(wc.openSessions)) + for _, info := range wc.openSessions { + openSessions = append(openSessions, info) + } + return openSessions +} + +func (wc *workflowEnvironmentImpl) GetRegistry() *registry { + return wc.registry +} + +// ResetLAWFTAttemptCounts resets the number of attempts in this WFT for all LAs to 0 - should be +// called at the beginning of every WFT +func (wc *workflowEnvironmentImpl) ResetLAWFTAttemptCounts() { + wc.completedLaAttemptsThisWFT = 0 + for _, task := range wc.pendingLaTasks { + task.Lock() + task.attemptsThisWFT = 0 + task.pastFirstWFT = true + task.Unlock() + } +} + +// GatherLAAttemptsThisWFT returns the total number of attempts in this WFT for all LAs who are +// past their first WFT +func (wc *workflowEnvironmentImpl) GatherLAAttemptsThisWFT() uint32 { + var attempts uint32 + for _, task := range wc.pendingLaTasks { + task.Lock() + if task.pastFirstWFT { + attempts += task.attemptsThisWFT + } + task.Unlock() + } + return attempts + wc.completedLaAttemptsThisWFT +} + +func (weh *workflowExecutionEventHandlerImpl) ProcessEvent( + event *historypb.HistoryEvent, + isReplay bool, + isLast bool, +) (err error) { + if event == nil { + return errors.New("nil event provided") + } + defer func() { + if p := recover(); p != nil { + weh.metricsHandler.Counter(metrics.WorkflowTaskExecutionFailureCounter).Inc(1) + topLine := fmt.Sprintf("process event for %s [panic]:", weh.workflowInfo.TaskQueueName) + st := getStackTraceRaw(topLine, 7, 0) + weh.Complete(nil, newWorkflowPanicError(p, st)) + } + }() + + weh.isReplay = isReplay + traceLog(func() { + weh.logger.Debug("ProcessEvent", + tagEventID, event.GetEventId(), + tagEventType, event.GetEventType().String()) + }) + + switch event.GetEventType() { + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED: + err = weh.handleWorkflowExecutionStarted(event.GetWorkflowExecutionStartedEventAttributes()) + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED: + // No Operation + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_FAILED: + // No Operation + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_TIMED_OUT: + // No Operation + case enumspb.EVENT_TYPE_WORKFLOW_TASK_SCHEDULED: + // No Operation + case enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED: + // Set replay clock. + weh.SetCurrentReplayTime(event.GetEventTime().AsTime()) + // Update workflow info fields + weh.workflowInfo.currentHistoryLength = int(event.EventId) + weh.workflowInfo.continueAsNewSuggested = event.GetWorkflowTaskStartedEventAttributes().GetSuggestContinueAsNew() + weh.workflowInfo.currentHistorySize = int(event.GetWorkflowTaskStartedEventAttributes().GetHistorySizeBytes()) + // Reset the counter on command helper used for generating ID for commands + weh.commandsHelper.setCurrentWorkflowTaskStartedEventID(event.GetEventId()) + weh.workflowDefinition.OnWorkflowTaskStarted(weh.deadlockDetectionTimeout) + + case enumspb.EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT: + // No Operation + case enumspb.EVENT_TYPE_WORKFLOW_TASK_FAILED: + // No Operation + case enumspb.EVENT_TYPE_WORKFLOW_TASK_COMPLETED: + // No Operation + case enumspb.EVENT_TYPE_ACTIVITY_TASK_SCHEDULED: + weh.commandsHelper.handleActivityTaskScheduled( + event.GetActivityTaskScheduledEventAttributes().GetActivityId(), event.GetEventId()) + + case enumspb.EVENT_TYPE_ACTIVITY_TASK_STARTED: + // No Operation + + case enumspb.EVENT_TYPE_ACTIVITY_TASK_COMPLETED: + err = weh.handleActivityTaskCompleted(event) + + case enumspb.EVENT_TYPE_ACTIVITY_TASK_FAILED: + err = weh.handleActivityTaskFailed(event) + + case enumspb.EVENT_TYPE_ACTIVITY_TASK_TIMED_OUT: + err = weh.handleActivityTaskTimedOut(event) + + case enumspb.EVENT_TYPE_ACTIVITY_TASK_CANCEL_REQUESTED: + weh.commandsHelper.handleActivityTaskCancelRequested( + event.GetActivityTaskCancelRequestedEventAttributes().GetScheduledEventId()) + + case enumspb.EVENT_TYPE_ACTIVITY_TASK_CANCELED: + err = weh.handleActivityTaskCanceled(event) + + case enumspb.EVENT_TYPE_TIMER_STARTED: + weh.commandsHelper.handleTimerStarted(event.GetTimerStartedEventAttributes().GetTimerId()) + + case enumspb.EVENT_TYPE_TIMER_FIRED: + weh.handleTimerFired(event) + + case enumspb.EVENT_TYPE_TIMER_CANCELED: + weh.commandsHelper.handleTimerCanceled(event.GetTimerCanceledEventAttributes().GetTimerId()) + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CANCEL_REQUESTED: + weh.handleWorkflowExecutionCancelRequested() + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CANCELED: + // No Operation. + + case enumspb.EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED: + _ = weh.handleRequestCancelExternalWorkflowExecutionInitiated(event) + + case enumspb.EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED: + _ = weh.handleRequestCancelExternalWorkflowExecutionFailed(event) + + case enumspb.EVENT_TYPE_EXTERNAL_WORKFLOW_EXECUTION_CANCEL_REQUESTED: + _ = weh.handleExternalWorkflowExecutionCancelRequested(event) + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_TERMINATED: + // No Operation + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW: + // No Operation. + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_SIGNALED: + err = weh.handleWorkflowExecutionSignaled(event.GetWorkflowExecutionSignaledEventAttributes()) + + case enumspb.EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED: + signalID := event.GetSignalExternalWorkflowExecutionInitiatedEventAttributes().Control + weh.commandsHelper.handleSignalExternalWorkflowExecutionInitiated(event.GetEventId(), signalID) + + case enumspb.EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED: + _ = weh.handleSignalExternalWorkflowExecutionFailed(event) + + case enumspb.EVENT_TYPE_EXTERNAL_WORKFLOW_EXECUTION_SIGNALED: + _ = weh.handleSignalExternalWorkflowExecutionCompleted(event) + + case enumspb.EVENT_TYPE_MARKER_RECORDED: + err = weh.handleMarkerRecorded(event.GetEventId(), event.GetMarkerRecordedEventAttributes()) + + case enumspb.EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_INITIATED: + weh.commandsHelper.handleStartChildWorkflowExecutionInitiated( + event.GetStartChildWorkflowExecutionInitiatedEventAttributes().GetWorkflowId()) + + case enumspb.EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_FAILED: + err = weh.handleStartChildWorkflowExecutionFailed(event) + + case enumspb.EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_STARTED: + err = weh.handleChildWorkflowExecutionStarted(event) + + case enumspb.EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_COMPLETED: + err = weh.handleChildWorkflowExecutionCompleted(event) + + case enumspb.EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_FAILED: + err = weh.handleChildWorkflowExecutionFailed(event) + + case enumspb.EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_CANCELED: + err = weh.handleChildWorkflowExecutionCanceled(event) + + case enumspb.EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_TIMED_OUT: + err = weh.handleChildWorkflowExecutionTimedOut(event) + + case enumspb.EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_TERMINATED: + err = weh.handleChildWorkflowExecutionTerminated(event) + + case enumspb.EVENT_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES: + weh.handleUpsertWorkflowSearchAttributes(event) + + case enumspb.EVENT_TYPE_WORKFLOW_PROPERTIES_MODIFIED: + weh.handleWorkflowPropertiesModified(event) + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_ADMITTED: + // No Operation + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_ACCEPTED: + // No Operation + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_REJECTED: + // No Operation + + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_COMPLETED: + // No Operation + + case enumspb.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED: + weh.commandsHelper.handleNexusOperationScheduled(event) + case enumspb.EVENT_TYPE_NEXUS_OPERATION_STARTED: + err = weh.handleNexusOperationStarted(event) + // all forms of completions are handled by the same method. + case enumspb.EVENT_TYPE_NEXUS_OPERATION_COMPLETED, + enumspb.EVENT_TYPE_NEXUS_OPERATION_FAILED, + enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCELED, + enumspb.EVENT_TYPE_NEXUS_OPERATION_TIMED_OUT: + err = weh.handleNexusOperationCompleted(event) + case enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUESTED: + weh.commandsHelper.handleNexusOperationCancelRequested(event.GetNexusOperationCancelRequestedEventAttributes().GetScheduledEventId()) + + default: + if event.WorkerMayIgnore { + // Do not fail to be forward compatible with new events + weh.logger.Debug("unknown event type", + tagEventID, event.GetEventId(), + tagEventType, event.GetEventType().String()) + } else { + weh.logger.Error("unknown event type", + tagEventID, event.GetEventId(), + tagEventType, event.GetEventType().String()) + return ErrUnknownHistoryEvent + } + } + + if err != nil { + return err + } + + // When replaying histories to get stack trace or current state the last event might be not + // workflow task started. So always call OnWorkflowTaskStarted on the last event. + // Don't call for EventType_WorkflowTaskStarted as it was already called when handling it. + if isLast && event.GetEventType() != enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED { + weh.workflowDefinition.OnWorkflowTaskStarted(weh.deadlockDetectionTimeout) + } + + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) ProcessMessage( + msg *protocolpb.Message, + isReplay bool, + isLast bool, +) error { + defer func() { + if p := recover(); p != nil { + weh.metricsHandler.Counter(metrics.WorkflowTaskExecutionFailureCounter).Inc(1) + topLine := fmt.Sprintf("process message for %s [panic]:", weh.workflowInfo.TaskQueueName) + st := getStackTraceRaw(topLine, 7, 0) + weh.Complete(nil, newWorkflowPanicError(p, st)) + } + }() + + ctor, err := weh.protocolConstructorForMessage(msg) + if err != nil { + return nil + } + instance := weh.protocols.FindOrAdd(msg.ProtocolInstanceId, ctor) + return instance.HandleMessage(msg) +} + +func (weh *workflowExecutionEventHandlerImpl) ProcessQuery( + queryType string, + queryArgs *commonpb.Payloads, + header *commonpb.Header, +) (*commonpb.Payloads, error) { + switch queryType { + case QueryTypeStackTrace: + return weh.encodeArg(weh.StackTrace()) + case QueryTypeOpenSessions: + return weh.encodeArg(weh.getOpenSessions()) + case QueryTypeWorkflowMetadata: + // We are intentionally not handling this here but rather in the + // normal handler so it has access to the options/context as + // needed. + fallthrough + default: + result, err := weh.queryHandler(queryType, queryArgs, header) + if err != nil { + return nil, err + } + + if result.Size() > queryResultSizeLimit { + weh.logger.Error("Query result size exceeds limit.", + tagQueryType, queryType, + tagWorkflowID, weh.workflowInfo.WorkflowExecution.ID, + tagRunID, weh.workflowInfo.WorkflowExecution.RunID) + return nil, fmt.Errorf("query result size (%v) exceeds limit (%v)", result.Size(), queryResultSizeLimit) + } + + return result, nil + } +} + +func (weh *workflowExecutionEventHandlerImpl) StackTrace() string { + return weh.workflowDefinition.StackTrace() +} + +func (weh *workflowExecutionEventHandlerImpl) Close() { + if weh.workflowDefinition != nil { + weh.workflowDefinition.Close() + } +} + +func (weh *workflowExecutionEventHandlerImpl) handleWorkflowExecutionStarted( + attributes *historypb.WorkflowExecutionStartedEventAttributes, +) (err error) { + weh.workflowDefinition, err = weh.registry.getWorkflowDefinition( + weh.workflowInfo.WorkflowType, + ) + if err != nil { + return err + } + + // We set this flag at workflow start because changing it on a mid-workflow + // WFT results in inconsistent values for SDKFlags during replay (i.e. + // replay sees the _final_ value of applied flags, not intermediate values + // as the value varies by WFT) + weh.sdkFlags.tryUse(SDKFlagProtocolMessageCommand, !weh.isReplay) + + // Invoke the workflow. + weh.workflowDefinition.Execute(weh, attributes.Header, attributes.Input) + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleActivityTaskCompleted(event *historypb.HistoryEvent) error { + activityID, scheduledEventID := weh.commandsHelper.getActivityAndScheduledEventIDs(event) + command := weh.commandsHelper.handleActivityTaskClosed(activityID, scheduledEventID) + activity := command.getData().(*scheduledActivity) + if activity.handled { + return nil + } + activity.handle(event.GetActivityTaskCompletedEventAttributes().Result, nil) + + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleActivityTaskFailed(event *historypb.HistoryEvent) error { + activityID, scheduledEventID := weh.commandsHelper.getActivityAndScheduledEventIDs(event) + command := weh.commandsHelper.handleActivityTaskClosed(activityID, scheduledEventID) + activity := command.getData().(*scheduledActivity) + if activity.handled { + return nil + } + + attributes := event.GetActivityTaskFailedEventAttributes() + activityTaskErr := NewActivityError( + attributes.GetScheduledEventId(), + attributes.GetStartedEventId(), + attributes.GetIdentity(), + &commonpb.ActivityType{Name: activity.activityType.Name}, + activityID, + attributes.GetRetryState(), + weh.GetFailureConverter().FailureToError(attributes.GetFailure()), + ) + + activity.handle(nil, activityTaskErr) + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleActivityTaskTimedOut(event *historypb.HistoryEvent) error { + activityID, scheduledEventID := weh.commandsHelper.getActivityAndScheduledEventIDs(event) + command := weh.commandsHelper.handleActivityTaskClosed(activityID, scheduledEventID) + activity := command.getData().(*scheduledActivity) + if activity.handled { + return nil + } + + attributes := event.GetActivityTaskTimedOutEventAttributes() + timeoutError := weh.GetFailureConverter().FailureToError(attributes.GetFailure()) + + activityTaskErr := NewActivityError( + attributes.GetScheduledEventId(), + attributes.GetStartedEventId(), + "", + &commonpb.ActivityType{Name: activity.activityType.Name}, + activityID, + attributes.GetRetryState(), + timeoutError, + ) + + activity.handle(nil, activityTaskErr) + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleActivityTaskCanceled(event *historypb.HistoryEvent) error { + activityID, scheduledEventID := weh.commandsHelper.getActivityAndScheduledEventIDs(event) + command := weh.commandsHelper.handleActivityTaskCanceled(activityID, scheduledEventID) + activity := command.getData().(*scheduledActivity) + if activity.handled { + return nil + } + + if command.isDone() || !activity.waitForCancelRequest { + // Clear this so we don't have a recursive call that while executing might call the cancel one. + + attributes := event.GetActivityTaskCanceledEventAttributes() + details := newEncodedValues(attributes.GetDetails(), weh.GetDataConverter()) + + activityTaskErr := NewActivityError( + attributes.GetScheduledEventId(), + attributes.GetStartedEventId(), + attributes.GetIdentity(), + &commonpb.ActivityType{Name: activity.activityType.Name}, + activityID, + enumspb.RETRY_STATE_NON_RETRYABLE_FAILURE, + NewCanceledError(details), + ) + + activity.handle(nil, activityTaskErr) + } + + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleTimerFired(event *historypb.HistoryEvent) { + timerID := event.GetTimerFiredEventAttributes().GetTimerId() + command := weh.commandsHelper.handleTimerClosed(timerID) + timer := command.getData().(*scheduledTimer) + if timer.handled { + return + } + + timer.handle(nil, nil) +} + +func (weh *workflowExecutionEventHandlerImpl) handleWorkflowExecutionCancelRequested() { + weh.cancelHandler() +} + +func (weh *workflowExecutionEventHandlerImpl) handleMarkerRecorded( + eventID int64, + attributes *historypb.MarkerRecordedEventAttributes, +) error { + var err error + if attributes.GetDetails() == nil { + err = ErrMissingMarkerDetails + } else { + switch attributes.GetMarkerName() { + case sideEffectMarkerName: + if sideEffectIDPayload, ok := attributes.GetDetails()[sideEffectMarkerIDName]; !ok { + err = fmt.Errorf("key %q: %w", sideEffectMarkerIDName, ErrMissingMarkerDataKey) + } else { + if sideEffectData, ok := attributes.GetDetails()[sideEffectMarkerDataName]; !ok { + err = fmt.Errorf("key %q: %w", sideEffectMarkerDataName, ErrMissingMarkerDataKey) + } else { + var sideEffectID int64 + _ = weh.dataConverter.FromPayloads(sideEffectIDPayload, &sideEffectID) + weh.sideEffectResult[sideEffectID] = sideEffectData + } + } + case versionMarkerName: + if changeIDPayload, ok := attributes.GetDetails()[versionMarkerChangeIDName]; !ok { + err = fmt.Errorf("key %q: %w", versionMarkerChangeIDName, ErrMissingMarkerDataKey) + } else { + if versionPayload, ok := attributes.GetDetails()[versionMarkerDataName]; !ok { + err = fmt.Errorf("key %q: %w", versionMarkerDataName, ErrMissingMarkerDataKey) + } else { + // versionSearchAttributeUpdatedName is optional and was only added later so do not expect all version + // markers to have this. + searchAttrUpdated := true + if searchAttrUpdatedPayload, ok := attributes.GetDetails()[versionSearchAttributeUpdatedName]; ok { + _ = weh.dataConverter.FromPayloads(searchAttrUpdatedPayload, &searchAttrUpdated) + } + var changeID string + _ = weh.dataConverter.FromPayloads(changeIDPayload, &changeID) + var version Version + _ = weh.dataConverter.FromPayloads(versionPayload, &version) + weh.changeVersions[changeID] = version + weh.commandsHelper.handleVersionMarker(eventID, changeID, searchAttrUpdated) + } + } + case localActivityMarkerName: + err = weh.handleLocalActivityMarker(attributes.GetDetails(), attributes.GetFailure()) + case mutableSideEffectMarkerName: + var sideEffectIDWithCounterPayload, sideEffectDataPayload *commonpb.Payloads + if sideEffectIDWithCounterPayload = attributes.GetDetails()[sideEffectMarkerIDName]; sideEffectIDWithCounterPayload == nil { + err = fmt.Errorf("key %q: %w", sideEffectMarkerIDName, ErrMissingMarkerDataKey) + } + if err == nil { + if sideEffectDataPayload = attributes.GetDetails()[sideEffectMarkerDataName]; sideEffectDataPayload == nil { + err = fmt.Errorf("key %q: %w", sideEffectMarkerDataName, ErrMissingMarkerDataKey) + } + } + var sideEffectIDWithCounter, sideEffectDataID string + var sideEffectDataContents commonpb.Payloads + if err == nil { + err = weh.dataConverter.FromPayloads(sideEffectIDWithCounterPayload, &sideEffectIDWithCounter) + } + // Side effect data is actually a wrapper of ID + data, so we need to + // extract the second value as the actual data + if err == nil { + err = weh.dataConverter.FromPayloads(sideEffectDataPayload, &sideEffectDataID, &sideEffectDataContents) + } + if err == nil { + counterHintPayload, ok := attributes.GetDetails()[mutableSideEffectCallCounterName] + var counterHint int + if ok { + err = weh.dataConverter.FromPayloads(counterHintPayload, &counterHint) + } else { + // An old version of the SDK did not write the counter hint so we have to assume. + // If multiple mutable side effects on the same ID are in a WFT only the last value is used. + counterHint = weh.mutableSideEffectCallCounter[sideEffectDataID] + } + if err == nil { + if weh.mutableSideEffect[sideEffectDataID] == nil { + weh.mutableSideEffect[sideEffectDataID] = make(map[int]*commonpb.Payloads) + } + weh.mutableSideEffect[sideEffectDataID][counterHint] = &sideEffectDataContents + // We must mark that it is recorded so we can know whether a command + // needs to be generated during replay + if weh.mutableSideEffectsRecorded == nil { + weh.mutableSideEffectsRecorded = map[string]bool{} + } + // This must be stored with the counter + weh.mutableSideEffectsRecorded[sideEffectIDWithCounter] = true + } + } + default: + err = ErrUnknownMarkerName + } + } + + if err != nil { + return fmt.Errorf("marker name %q for eventId %d: %w", attributes.GetMarkerName(), eventID, err) + } + + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleLocalActivityMarker(details map[string]*commonpb.Payloads, failure *failurepb.Failure) error { + var markerData *commonpb.Payloads + var ok bool + if markerData, ok = details[localActivityMarkerDataName]; !ok { + return fmt.Errorf("key %q: %w", localActivityMarkerDataName, ErrMissingMarkerDataKey) + } + + lamd := localActivityMarkerData{} + if err := weh.dataConverter.FromPayloads(markerData, &lamd); err != nil { + return err + } + + if la, ok := weh.pendingLaTasks[lamd.ActivityID]; ok { + if len(lamd.ActivityType) > 0 && lamd.ActivityType != la.params.ActivityType { + // history marker mismatch to the current code. + panicMsg := fmt.Sprintf("[TMPRL1100] code execute local activity %v, but history event found %v, markerData: %v", la.params.ActivityType, lamd.ActivityType, markerData) + panicIllegalState(panicMsg) + } + weh.commandsHelper.recordLocalActivityMarker(lamd.ActivityID, details, failure) + if la.pastFirstWFT { + weh.completedLaAttemptsThisWFT += la.attemptsThisWFT + } + delete(weh.pendingLaTasks, lamd.ActivityID) + delete(weh.unstartedLaTasks, lamd.ActivityID) + lar := &LocalActivityResultWrapper{} + if failure != nil { + lar.Attempt = lamd.Attempt + lar.Backoff = lamd.Backoff + lar.Err = weh.GetFailureConverter().FailureToError(failure) + } else { + // Result might not be there if local activity doesn't have return value. + lar.Result = details[localActivityResultName] + } + la.callback(lar) + + // update time + weh.SetCurrentReplayTime(lamd.ReplayTime) + + // resume workflow execution after apply local activity result + weh.workflowDefinition.OnWorkflowTaskStarted(weh.deadlockDetectionTimeout) + } + + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) ProcessLocalActivityResult(lar *localActivityResult) error { + details := make(map[string]*commonpb.Payloads) + + // convert local activity result and error to marker data + lamd := localActivityMarkerData{ + ActivityID: lar.task.activityID, + ActivityType: lar.task.params.ActivityType, + ReplayTime: weh.currentReplayTime.Add(time.Since(weh.currentLocalTime)), + Attempt: lar.task.attempt, + } + if lar.err != nil { + lamd.Backoff = lar.backoff + } else if lar.result != nil { + details[localActivityResultName] = lar.result + } + + // encode marker data + markerData, err := weh.encodeArg(lamd) + if err != nil { + return err + } + details[localActivityMarkerDataName] = markerData + + // create marker event for local activity result + markerEvent := &historypb.HistoryEvent{ + EventType: enumspb.EVENT_TYPE_MARKER_RECORDED, + Attributes: &historypb.HistoryEvent_MarkerRecordedEventAttributes{MarkerRecordedEventAttributes: &historypb.MarkerRecordedEventAttributes{ + MarkerName: localActivityMarkerName, + Failure: weh.GetFailureConverter().ErrorToFailure(lar.err), + Details: details, + }}, + } + + // apply the local activity result to workflow + return weh.ProcessEvent(markerEvent, false, false) +} + +func (weh *workflowExecutionEventHandlerImpl) handleWorkflowExecutionSignaled( + attributes *historypb.WorkflowExecutionSignaledEventAttributes, +) error { + return weh.signalHandler(attributes.GetSignalName(), attributes.Input, attributes.Header) +} + +func (weh *workflowExecutionEventHandlerImpl) handleStartChildWorkflowExecutionFailed(event *historypb.HistoryEvent) error { + attributes := event.GetStartChildWorkflowExecutionFailedEventAttributes() + childWorkflowID := attributes.GetWorkflowId() + command := weh.commandsHelper.handleStartChildWorkflowExecutionFailed(childWorkflowID) + childWorkflow := command.getData().(*scheduledChildWorkflow) + if childWorkflow.handled { + return nil + } + + var causeErr error + switch attributes.GetCause() { + case enumspb.START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_WORKFLOW_ALREADY_EXISTS: + causeErr = &ChildWorkflowExecutionAlreadyStartedError{} + case enumspb.START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND: + causeErr = &NamespaceNotFoundError{} + default: + causeErr = fmt.Errorf("unable to start child workflow for unknown cause: %v", attributes.GetCause()) + } + + err := NewChildWorkflowExecutionError( + attributes.GetNamespace(), + attributes.GetWorkflowId(), + "", + attributes.GetWorkflowType().GetName(), + attributes.GetInitiatedEventId(), + 0, + enumspb.RETRY_STATE_NON_RETRYABLE_FAILURE, + causeErr, + ) + childWorkflow.handleFailedToStart(nil, err) + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleChildWorkflowExecutionStarted(event *historypb.HistoryEvent) error { + attributes := event.GetChildWorkflowExecutionStartedEventAttributes() + childWorkflowID := attributes.WorkflowExecution.GetWorkflowId() + childRunID := attributes.WorkflowExecution.GetRunId() + command := weh.commandsHelper.handleChildWorkflowExecutionStarted(childWorkflowID) + childWorkflow := command.getData().(*scheduledChildWorkflow) + if childWorkflow.handled { + return nil + } + + childWorkflowExecution := WorkflowExecution{ + ID: childWorkflowID, + RunID: childRunID, + } + childWorkflow.startedCallback(childWorkflowExecution, nil) + + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleChildWorkflowExecutionCompleted(event *historypb.HistoryEvent) error { + attributes := event.GetChildWorkflowExecutionCompletedEventAttributes() + childWorkflowID := attributes.WorkflowExecution.GetWorkflowId() + command := weh.commandsHelper.handleChildWorkflowExecutionClosed(childWorkflowID) + childWorkflow := command.getData().(*scheduledChildWorkflow) + if childWorkflow.handled { + return nil + } + childWorkflow.handle(attributes.Result, nil) + + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleChildWorkflowExecutionFailed(event *historypb.HistoryEvent) error { + attributes := event.GetChildWorkflowExecutionFailedEventAttributes() + childWorkflowID := attributes.WorkflowExecution.GetWorkflowId() + command := weh.commandsHelper.handleChildWorkflowExecutionClosed(childWorkflowID) + childWorkflow := command.getData().(*scheduledChildWorkflow) + if childWorkflow.handled { + return nil + } + + childWorkflowExecutionError := NewChildWorkflowExecutionError( + attributes.GetNamespace(), + attributes.GetWorkflowExecution().GetWorkflowId(), + attributes.GetWorkflowExecution().GetRunId(), + attributes.GetWorkflowType().GetName(), + attributes.GetInitiatedEventId(), + attributes.GetStartedEventId(), + attributes.GetRetryState(), + weh.GetFailureConverter().FailureToError(attributes.GetFailure()), + ) + childWorkflow.handle(nil, childWorkflowExecutionError) + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleChildWorkflowExecutionCanceled(event *historypb.HistoryEvent) error { + attributes := event.GetChildWorkflowExecutionCanceledEventAttributes() + childWorkflowID := attributes.WorkflowExecution.GetWorkflowId() + command := weh.commandsHelper.handleChildWorkflowExecutionCanceled(childWorkflowID) + childWorkflow := command.getData().(*scheduledChildWorkflow) + if childWorkflow.handled { + return nil + } + details := newEncodedValues(attributes.Details, weh.GetDataConverter()) + + childWorkflowExecutionError := NewChildWorkflowExecutionError( + attributes.GetNamespace(), + attributes.GetWorkflowExecution().GetWorkflowId(), + attributes.GetWorkflowExecution().GetRunId(), + attributes.GetWorkflowType().GetName(), + attributes.GetInitiatedEventId(), + attributes.GetStartedEventId(), + enumspb.RETRY_STATE_NON_RETRYABLE_FAILURE, + NewCanceledError(details), + ) + childWorkflow.handle(nil, childWorkflowExecutionError) + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleChildWorkflowExecutionTimedOut(event *historypb.HistoryEvent) error { + attributes := event.GetChildWorkflowExecutionTimedOutEventAttributes() + childWorkflowID := attributes.WorkflowExecution.GetWorkflowId() + command := weh.commandsHelper.handleChildWorkflowExecutionClosed(childWorkflowID) + childWorkflow := command.getData().(*scheduledChildWorkflow) + if childWorkflow.handled { + return nil + } + + childWorkflowExecutionError := NewChildWorkflowExecutionError( + attributes.GetNamespace(), + attributes.GetWorkflowExecution().GetWorkflowId(), + attributes.GetWorkflowExecution().GetRunId(), + attributes.GetWorkflowType().GetName(), + attributes.GetInitiatedEventId(), + attributes.GetStartedEventId(), + attributes.GetRetryState(), + NewTimeoutError("Child workflow timeout", enumspb.TIMEOUT_TYPE_START_TO_CLOSE, nil), + ) + childWorkflow.handle(nil, childWorkflowExecutionError) + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleChildWorkflowExecutionTerminated(event *historypb.HistoryEvent) error { + attributes := event.GetChildWorkflowExecutionTerminatedEventAttributes() + childWorkflowID := attributes.WorkflowExecution.GetWorkflowId() + command := weh.commandsHelper.handleChildWorkflowExecutionClosed(childWorkflowID) + childWorkflow := command.getData().(*scheduledChildWorkflow) + if childWorkflow.handled { + return nil + } + + childWorkflowExecutionError := NewChildWorkflowExecutionError( + attributes.GetNamespace(), + attributes.GetWorkflowExecution().GetWorkflowId(), + attributes.GetWorkflowExecution().GetRunId(), + attributes.GetWorkflowType().GetName(), + attributes.GetInitiatedEventId(), + attributes.GetStartedEventId(), + enumspb.RETRY_STATE_NON_RETRYABLE_FAILURE, + newTerminatedError(), + ) + childWorkflow.handle(nil, childWorkflowExecutionError) + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleNexusOperationStarted(event *historypb.HistoryEvent) error { + attributes := event.GetNexusOperationStartedEventAttributes() + command := weh.commandsHelper.handleNexusOperationStarted(attributes.ScheduledEventId) + state := command.getData().(*scheduledNexusOperation) + if state.startedCallback != nil { + state.startedCallback(attributes.OperationId, nil) + state.startedCallback = nil + } + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleNexusOperationCompleted(event *historypb.HistoryEvent) error { + var result *commonpb.Payload + var failure *failurepb.Failure + var scheduledEventId int64 + + switch event.EventType { + case enumspb.EVENT_TYPE_NEXUS_OPERATION_COMPLETED: + attrs := event.GetNexusOperationCompletedEventAttributes() + result = attrs.GetResult() + scheduledEventId = attrs.GetScheduledEventId() + case enumspb.EVENT_TYPE_NEXUS_OPERATION_FAILED: + attrs := event.GetNexusOperationFailedEventAttributes() + failure = attrs.GetFailure() + scheduledEventId = attrs.GetScheduledEventId() + case enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCELED: + attrs := event.GetNexusOperationCanceledEventAttributes() + failure = attrs.GetFailure() + scheduledEventId = attrs.GetScheduledEventId() + case enumspb.EVENT_TYPE_NEXUS_OPERATION_TIMED_OUT: + attrs := event.GetNexusOperationTimedOutEventAttributes() + failure = attrs.GetFailure() + scheduledEventId = attrs.GetScheduledEventId() + default: + // This is only called internally and should never happen. + panic(fmt.Errorf("invalid event type, not a Nexus Operation resolution: %v", event.EventType)) + } + command := weh.commandsHelper.handleNexusOperationCompleted(scheduledEventId) + state := command.getData().(*scheduledNexusOperation) + var err error + if failure != nil { + err = weh.failureConverter.FailureToError(failure) + } + // Also unblock the start future + if state.startedCallback != nil { + state.startedCallback("", err) // We didn't get a started event, the operation completed synchronously. + state.startedCallback = nil + } + if state.completedCallback != nil { + state.completedCallback(result, err) + state.completedCallback = nil + } + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleUpsertWorkflowSearchAttributes(event *historypb.HistoryEvent) { + weh.updateWorkflowInfoWithSearchAttributes(event.GetUpsertWorkflowSearchAttributesEventAttributes().SearchAttributes) +} + +func (weh *workflowExecutionEventHandlerImpl) handleWorkflowPropertiesModified( + event *historypb.HistoryEvent, +) { + attributes := event.GetWorkflowPropertiesModifiedEventAttributes() + weh.updateWorkflowInfoWithMemo(attributes.UpsertedMemo) +} + +func (weh *workflowExecutionEventHandlerImpl) handleRequestCancelExternalWorkflowExecutionInitiated(event *historypb.HistoryEvent) error { + // For cancellation of child workflow only, we do not use cancellation ID + // for cancellation of external workflow, we have to use cancellation ID + attribute := event.GetRequestCancelExternalWorkflowExecutionInitiatedEventAttributes() + workflowID := attribute.WorkflowExecution.GetWorkflowId() + cancellationID := attribute.Control + weh.commandsHelper.handleRequestCancelExternalWorkflowExecutionInitiated(event.GetEventId(), workflowID, cancellationID) + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleExternalWorkflowExecutionCancelRequested(event *historypb.HistoryEvent) error { + // For cancellation of child workflow only, we do not use cancellation ID + // for cancellation of external workflow, we have to use cancellation ID + attributes := event.GetExternalWorkflowExecutionCancelRequestedEventAttributes() + workflowID := attributes.WorkflowExecution.GetWorkflowId() + isExternal, command := weh.commandsHelper.handleExternalWorkflowExecutionCancelRequested(attributes.GetInitiatedEventId(), workflowID) + if isExternal { + // for cancel external workflow, we need to set the future + cancellation := command.getData().(*scheduledCancellation) + if cancellation.handled { + return nil + } + cancellation.handle(nil, nil) + } + + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleRequestCancelExternalWorkflowExecutionFailed(event *historypb.HistoryEvent) error { + // For cancellation of child workflow only, we do not use cancellation ID + // for cancellation of external workflow, we have to use cancellation ID + attributes := event.GetRequestCancelExternalWorkflowExecutionFailedEventAttributes() + workflowID := attributes.WorkflowExecution.GetWorkflowId() + isExternal, command := weh.commandsHelper.handleRequestCancelExternalWorkflowExecutionFailed(attributes.GetInitiatedEventId(), workflowID) + if isExternal { + // for cancel external workflow, we need to set the future + cancellation := command.getData().(*scheduledCancellation) + if cancellation.handled { + return nil + } + + var err error + switch attributes.GetCause() { + case enumspb.CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_EXTERNAL_WORKFLOW_EXECUTION_NOT_FOUND: + err = newUnknownExternalWorkflowExecutionError() + case enumspb.CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND: + err = &NamespaceNotFoundError{} + default: + err = fmt.Errorf("unable to cancel external workflow for unknown cause: %v", attributes.GetCause()) + } + cancellation.handle(nil, err) + } + + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleSignalExternalWorkflowExecutionCompleted(event *historypb.HistoryEvent) error { + attributes := event.GetExternalWorkflowExecutionSignaledEventAttributes() + command := weh.commandsHelper.handleSignalExternalWorkflowExecutionCompleted(attributes.GetInitiatedEventId()) + signal := command.getData().(*scheduledSignal) + if signal.handled { + return nil + } + signal.handle(nil, nil) + + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleSignalExternalWorkflowExecutionFailed(event *historypb.HistoryEvent) error { + attributes := event.GetSignalExternalWorkflowExecutionFailedEventAttributes() + command := weh.commandsHelper.handleSignalExternalWorkflowExecutionFailed(attributes.GetInitiatedEventId()) + signal := command.getData().(*scheduledSignal) + if signal.handled { + return nil + } + + var err error + switch attributes.GetCause() { + case enumspb.SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_EXTERNAL_WORKFLOW_EXECUTION_NOT_FOUND: + err = newUnknownExternalWorkflowExecutionError() + case enumspb.SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_NAMESPACE_NOT_FOUND: + err = &NamespaceNotFoundError{} + default: + err = fmt.Errorf("unable to signal external workflow for unknown cause: %v", attributes.GetCause()) + } + + signal.handle(nil, err) + + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) protocolConstructorForMessage( + msg *protocolpb.Message, +) (func() protocol.Instance, error) { + protoName, err := protocol.NameFromMessage(msg) + if err != nil { + return nil, err + } + + switch protoName { + case updateProtocolV1: + return func() protocol.Instance { + return newUpdateProtocol(msg.ProtocolInstanceId, weh.updateHandler, weh) + }, nil + } + return nil, fmt.Errorf("unsupported protocol: %v", protoName) +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_flags.go b/vendor/go.temporal.io/sdk/internal/internal_flags.go new file mode 100644 index 00000000000..503c650a51a --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_flags.go @@ -0,0 +1,133 @@ +// The MIT License +// +// Copyright (c) 2023 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "math" + + "go.temporal.io/api/workflowservice/v1" +) + +// sdkFlag represents a flag used to help version the sdk internally to make breaking changes +// in workflow logic. +type sdkFlag uint32 + +const ( + SDKFlagUnset sdkFlag = 0 + // LimitChangeVersionSASize will limit the search attribute size of TemporalChangeVersion to 2048 when + // calling GetVersion. If the limit is exceeded the search attribute is not updated. + SDKFlagLimitChangeVersionSASize = 1 + // SDKFlagChildWorkflowErrorExecution return errors to child workflow execution future if the child workflow would + // fail in the synchronous path. + SDKFlagChildWorkflowErrorExecution = 2 + // SDKFlagProtocolMessageCommand uses ProtocolMessageCommands inserted into + // a workflow task response's command set to order messages with respect to + // commands. + SDKFlagProtocolMessageCommand = 3 + // SDKPriorityUpdateHandling will cause update request to be handled before the main workflow method. + // It will also cause the SDK to immediately handle updates when a handler is registered. + SDKPriorityUpdateHandling = 4 + SDKFlagUnknown = math.MaxUint32 +) + +func sdkFlagFromUint(value uint32) sdkFlag { + switch value { + case uint32(SDKFlagUnset): + return SDKFlagUnset + case uint32(SDKFlagLimitChangeVersionSASize): + return SDKFlagLimitChangeVersionSASize + case uint32(SDKFlagChildWorkflowErrorExecution): + return SDKFlagChildWorkflowErrorExecution + case uint32(SDKFlagProtocolMessageCommand): + return SDKFlagProtocolMessageCommand + case uint32(SDKPriorityUpdateHandling): + return SDKPriorityUpdateHandling + default: + return SDKFlagUnknown + } +} + +func (f sdkFlag) isValid() bool { + return f != SDKFlagUnset && f != SDKFlagUnknown +} + +// sdkFlags represents all the flags that are currently set in a workflow execution. +type sdkFlags struct { + capabilities *workflowservice.GetSystemInfoResponse_Capabilities + // Flags that have been recieved from the server + currentFlags map[sdkFlag]bool + // Flags that have been set this WFT that have not been sent to the server. + // Keep track of them sepratly so we know what to send to the server. + newFlags map[sdkFlag]bool +} + +func newSDKFlags(capabilities *workflowservice.GetSystemInfoResponse_Capabilities) *sdkFlags { + return &sdkFlags{ + capabilities: capabilities, + currentFlags: make(map[sdkFlag]bool), + newFlags: make(map[sdkFlag]bool), + } +} + +// tryUse returns true if this flag may currently be used. If record is true, always returns +// true and records the flag as being used. +func (sf *sdkFlags) tryUse(flag sdkFlag, record bool) bool { + if !sf.capabilities.GetSdkMetadata() { + return false + } + + if record && !sf.currentFlags[flag] { + // Only set new flags + sf.newFlags[flag] = true + return true + } else { + return sf.currentFlags[flag] + } +} + +// set marks a flag as in current use regardless of replay status. +func (sf *sdkFlags) set(flags ...sdkFlag) { + if !sf.capabilities.GetSdkMetadata() { + return + } + for _, flag := range flags { + sf.currentFlags[flag] = true + } +} + +// markSDKFlagsSent marks all sdk flags as sent to the server. +func (sf *sdkFlags) markSDKFlagsSent() { + for flag := range sf.newFlags { + sf.currentFlags[flag] = true + } + sf.newFlags = make(map[sdkFlag]bool) +} + +// gatherNewSDKFlags returns all sdkFlags set since the last call to markSDKFlagsSent. +func (sf *sdkFlags) gatherNewSDKFlags() []sdkFlag { + flags := make([]sdkFlag, 0, len(sf.newFlags)) + for flag := range sf.newFlags { + flags = append(flags, flag) + } + return flags +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_logging_tags.go b/vendor/go.temporal.io/sdk/internal/internal_logging_tags.go new file mode 100644 index 00000000000..1d49ce8ff26 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_logging_tags.go @@ -0,0 +1,58 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +const ( + tagActivityID = "ActivityID" + tagActivityType = "ActivityType" + tagNamespace = "Namespace" + tagEventID = "EventID" + tagEventType = "EventType" + tagRunID = "RunID" + tagTaskQueue = "TaskQueue" + tagTimerID = "TimerID" + tagWorkflowID = "WorkflowID" + tagWorkflowType = "WorkflowType" + tagWorkerID = "WorkerID" + tagBuildID = "BuildID" + tagWorkerType = "WorkerType" + tagSideEffectID = "SideEffectID" + tagChildWorkflowID = "ChildWorkflowID" + tagLocalActivityType = "LocalActivityType" + tagQueryType = "QueryType" + tagResult = "Result" + tagError = "Error" + tagStackTrace = "StackTrace" + tagAttempt = "Attempt" + tagTaskFirstEventID = "TaskFirstEventID" + tagTaskStartedEventID = "TaskStartedEventID" + tagPreviousStartedEventID = "PreviousStartedEventID" + tagCachedPreviousStartedEventID = "CachedPreviousStartedEventID" + tagNexusEndpoint = "NexusEndpoint" + tagNexusOperation = "NexusOperation" + tagNexusService = "NexusService" + tagPanicError = "PanicError" + tagPanicStack = "PanicStack" +) diff --git a/vendor/go.temporal.io/sdk/internal/internal_message.go b/vendor/go.temporal.io/sdk/internal/internal_message.go new file mode 100644 index 00000000000..4831fba3ca8 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_message.go @@ -0,0 +1,57 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + protocolpb "go.temporal.io/api/protocol/v1" +) + +type eventMsgIndex []*protocolpb.Message + +// indexMessagesByEventID creates an index over a set of input messages that +// allows for access to messages with an event ID less than or equal to a +// specific upper bound. The order of messages with the same event ID will be +// preserved. +func indexMessagesByEventID(msgs []*protocolpb.Message) *eventMsgIndex { + emi := eventMsgIndex(msgs) + return &emi +} + +// takeLTE removes and returns the messages in this index that have an event ID +// less than or equal to the input argument. +func (emi *eventMsgIndex) takeLTE(eventID int64) []*protocolpb.Message { + n := 0 + var out []*protocolpb.Message + for _, msg := range *emi { + if msg.GetEventId() > eventID { + (*emi)[n] = msg + n++ + } else { + out = append(out, msg) + } + } + *emi = (*emi)[:n] + return out +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_nexus_task_handler.go b/vendor/go.temporal.io/sdk/internal/internal_nexus_task_handler.go new file mode 100644 index 00000000000..03e3f5387e2 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_nexus_task_handler.go @@ -0,0 +1,484 @@ +// The MIT License +// +// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "bytes" + "context" + "errors" + "fmt" + "io" + "net/url" + "reflect" + "runtime/debug" + "time" + + "github.com/nexus-rpc/sdk-go/nexus" + "go.temporal.io/api/common/v1" + nexuspb "go.temporal.io/api/nexus/v1" + "go.temporal.io/api/workflowservice/v1" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/log" +) + +func nexusHandlerError(t nexus.HandlerErrorType, message string) *nexuspb.HandlerError { + return &nexuspb.HandlerError{ + ErrorType: string(t), + Failure: &nexuspb.Failure{ + Message: message, + }, + } +} + +func nexusHandlerErrorToProto(handlerErr *nexus.HandlerError) *nexuspb.HandlerError { + pbHandlerErr := &nexuspb.HandlerError{ + ErrorType: string(handlerErr.Type), + } + if handlerErr.Failure != nil { + pbHandlerErr.Failure = &nexuspb.Failure{ + Message: handlerErr.Failure.Message, + Metadata: handlerErr.Failure.Metadata, + Details: handlerErr.Failure.Details, + } + } + return pbHandlerErr +} + +type nexusTaskHandler struct { + nexusHandler nexus.Handler + identity string + namespace string + taskQueueName string + client Client + dataConverter converter.DataConverter + logger log.Logger + metricsHandler metrics.Handler +} + +func newNexusTaskHandler( + nexusHandler nexus.Handler, + identity string, + namespace string, + taskQueueName string, + client Client, + dataConverter converter.DataConverter, + logger log.Logger, + metricsHandler metrics.Handler, +) *nexusTaskHandler { + return &nexusTaskHandler{ + nexusHandler: nexusHandler, + logger: logger, + dataConverter: dataConverter, + identity: identity, + namespace: namespace, + taskQueueName: taskQueueName, + client: client, + metricsHandler: metricsHandler, + } +} + +func (h *nexusTaskHandler) Execute(task *workflowservice.PollNexusTaskQueueResponse) (*workflowservice.RespondNexusTaskCompletedRequest, *workflowservice.RespondNexusTaskFailedRequest, error) { + res, handlerErr, err := h.execute(task) + if err != nil { + return nil, nil, err + } + if handlerErr != nil { + return nil, h.fillInFailure(task.TaskToken, handlerErr), nil + } + return h.fillInCompletion(task.TaskToken, res), nil, nil +} + +func (h *nexusTaskHandler) execute(task *workflowservice.PollNexusTaskQueueResponse) (*nexuspb.Response, *nexuspb.HandlerError, error) { + metricsHandler, handlerErr := h.metricsHandlerForTask(task) + if handlerErr != nil { + return nil, handlerErr, nil + } + log, handlerErr := h.loggerForTask(task) + if handlerErr != nil { + return nil, handlerErr, nil + } + nctx := &NexusOperationContext{ + Client: h.client, + Namespace: h.namespace, + TaskQueue: h.taskQueueName, + MetricsHandler: metricsHandler, + Log: log, + } + header := nexus.Header(task.GetRequest().GetHeader()) + if header == nil { + header = nexus.Header{} + } + + ctx, cancel, handlerErr := h.goContextForTask(nctx, header) + if handlerErr != nil { + return nil, handlerErr, nil + } + defer cancel() + + switch req := task.GetRequest().GetVariant().(type) { + case *nexuspb.Request_StartOperation: + return h.handleStartOperation(ctx, nctx, req.StartOperation, header) + case *nexuspb.Request_CancelOperation: + return h.handleCancelOperation(ctx, nctx, req.CancelOperation, header) + default: + return nil, nexusHandlerError(nexus.HandlerErrorTypeNotImplemented, "unknown request type"), nil + } +} + +func (h *nexusTaskHandler) handleStartOperation( + ctx context.Context, + nctx *NexusOperationContext, + req *nexuspb.StartOperationRequest, + header nexus.Header, +) (*nexuspb.Response, *nexuspb.HandlerError, error) { + serializer := &payloadSerializer{ + converter: h.dataConverter, + payload: req.GetPayload(), + } + // Create a fake lazy value, Temporal server already converts Nexus content into payloads. + input := nexus.NewLazyValue( + serializer, + &nexus.Reader{ + ReadCloser: emptyReaderNopCloser, + }, + ) + // Ensure we don't pass nil values to handlers. + callbackHeader := req.GetCallbackHeader() + if callbackHeader == nil { + callbackHeader = make(map[string]string) + } + nexusLinks := make([]nexus.Link, 0, len(req.GetLinks())) + for _, link := range req.GetLinks() { + if link == nil { + continue + } + linkURL, err := url.Parse(link.GetUrl()) + if err != nil { + nctx.Log.Error("failed to parse link url: %s", link.GetUrl(), tagError, err) + return nil, nexusHandlerError(nexus.HandlerErrorTypeBadRequest, "failed to parse link url"), nil + } + nexusLinks = append(nexusLinks, nexus.Link{ + URL: linkURL, + Type: link.GetType(), + }) + } + startOptions := nexus.StartOperationOptions{ + RequestID: req.RequestId, + CallbackURL: req.Callback, + Header: header, + CallbackHeader: callbackHeader, + Links: nexusLinks, + } + var opres nexus.HandlerStartOperationResult[any] + var err error + func() { + defer func() { + recovered := recover() + if recovered != nil { + var ok bool + err, ok = recovered.(error) + if !ok { + err = fmt.Errorf("panic: %v", recovered) + } + + nctx.Log.Error("Panic captured while handling nexus task", tagStackTrace, string(debug.Stack()), tagError, err) + } + }() + opres, err = h.nexusHandler.StartOperation(ctx, req.GetService(), req.GetOperation(), input, startOptions) + }() + if ctx.Err() != nil { + return nil, nil, ctx.Err() + } + if err != nil { + var unsuccessfulOperationErr *nexus.UnsuccessfulOperationError + if errors.As(err, &unsuccessfulOperationErr) { + return &nexuspb.Response{ + Variant: &nexuspb.Response_StartOperation{ + StartOperation: &nexuspb.StartOperationResponse{ + Variant: &nexuspb.StartOperationResponse_OperationError{ + OperationError: &nexuspb.UnsuccessfulOperationError{ + OperationState: string(unsuccessfulOperationErr.State), + Failure: &nexuspb.Failure{ + Message: unsuccessfulOperationErr.Failure.Message, + Metadata: unsuccessfulOperationErr.Failure.Metadata, + Details: unsuccessfulOperationErr.Failure.Details, + }, + }, + }, + }, + }, + }, nil, nil + } + err = convertKnownErrors(err) + var handlerErr *nexus.HandlerError + if errors.As(err, &handlerErr) { + return nil, nexusHandlerErrorToProto(handlerErr), nil + } + // Default to internal error. + return nil, h.internalError(err), nil + } + switch t := opres.(type) { + case *nexus.HandlerStartOperationResultAsync: + var links []*nexuspb.Link + for _, nexusLink := range t.Links { + links = append(links, &nexuspb.Link{ + Url: nexusLink.URL.String(), + Type: nexusLink.Type, + }) + } + return &nexuspb.Response{ + Variant: &nexuspb.Response_StartOperation{ + StartOperation: &nexuspb.StartOperationResponse{ + Variant: &nexuspb.StartOperationResponse_AsyncSuccess{ + AsyncSuccess: &nexuspb.StartOperationResponse_Async{ + OperationId: t.OperationID, + Links: links, + }, + }, + }, + }, + }, nil, nil + default: + // *nexus.HandlerStartOperationResultSync is generic, we can't type switch unfortunately. + value := reflect.ValueOf(t).Elem().FieldByName("Value").Interface() + payload, err := h.dataConverter.ToPayload(value) + if err != nil { + return nil, h.internalError(fmt.Errorf("cannot convert nexus sync result: %w", err)), nil + } + return &nexuspb.Response{ + Variant: &nexuspb.Response_StartOperation{ + StartOperation: &nexuspb.StartOperationResponse{ + Variant: &nexuspb.StartOperationResponse_SyncSuccess{ + SyncSuccess: &nexuspb.StartOperationResponse_Sync{ + Payload: payload, + }, + }, + }, + }, + }, nil, nil + } +} + +func (h *nexusTaskHandler) handleCancelOperation(ctx context.Context, nctx *NexusOperationContext, req *nexuspb.CancelOperationRequest, header nexus.Header) (*nexuspb.Response, *nexuspb.HandlerError, error) { + cancelOptions := nexus.CancelOperationOptions{Header: header} + var err error + func() { + defer func() { + recovered := recover() + if recovered != nil { + var ok bool + err, ok = recovered.(error) + if !ok { + err = fmt.Errorf("panic: %v", recovered) + } + + nctx.Log.Error("Panic captured while handling nexus task", tagStackTrace, string(debug.Stack()), tagError, err) + } + }() + err = h.nexusHandler.CancelOperation(ctx, req.GetService(), req.GetOperation(), req.GetOperationId(), cancelOptions) + }() + if ctx.Err() != nil { + return nil, nil, ctx.Err() + } + if err != nil { + err = convertKnownErrors(err) + var handlerErr *nexus.HandlerError + if errors.As(err, &handlerErr) { + return nil, nexusHandlerErrorToProto(handlerErr), nil + } + // Default to internal error. + return nil, h.internalError(err), nil + } + + return &nexuspb.Response{ + Variant: &nexuspb.Response_CancelOperation{ + CancelOperation: &nexuspb.CancelOperationResponse{}, + }, + }, nil, nil +} + +func (h *nexusTaskHandler) internalError(err error) *nexuspb.HandlerError { + h.logger.Error("error processing nexus task", "error", err) + return nexusHandlerError(nexus.HandlerErrorTypeInternal, err.Error()) +} + +func (h *nexusTaskHandler) goContextForTask(nctx *NexusOperationContext, header nexus.Header) (context.Context, context.CancelFunc, *nexuspb.HandlerError) { + // Associate the NexusOperationContext with the context.Context used to invoke operations. + ctx := context.WithValue(context.Background(), nexusOperationContextKey, nctx) + + timeoutStr := header.Get(nexus.HeaderRequestTimeout) + if timeoutStr != "" { + timeout, err := time.ParseDuration(timeoutStr) + if err != nil { + return nil, nil, nexusHandlerError(nexus.HandlerErrorTypeBadRequest, "cannot parse request timeout") + } + ctx, cancel := context.WithTimeout(ctx, timeout) + return ctx, cancel, nil + } + + return ctx, func() {}, nil +} + +func (h *nexusTaskHandler) loggerForTask(response *workflowservice.PollNexusTaskQueueResponse) (log.Logger, *nexuspb.HandlerError) { + var service, operation string + + switch req := response.GetRequest().GetVariant().(type) { + case *nexuspb.Request_StartOperation: + service = req.StartOperation.Service + operation = req.StartOperation.Operation + case *nexuspb.Request_CancelOperation: + service = req.CancelOperation.Service + operation = req.CancelOperation.Operation + default: + return nil, nexusHandlerError(nexus.HandlerErrorTypeNotImplemented, "unknown request type") + } + + return log.With(h.logger, + tagNexusService, service, + tagNexusOperation, operation, + tagTaskQueue, h.taskQueueName, + ), nil +} + +func (h *nexusTaskHandler) metricsHandlerForTask(response *workflowservice.PollNexusTaskQueueResponse) (metrics.Handler, *nexuspb.HandlerError) { + var service, operation string + + switch req := response.GetRequest().GetVariant().(type) { + case *nexuspb.Request_StartOperation: + service = req.StartOperation.Service + operation = req.StartOperation.Operation + case *nexuspb.Request_CancelOperation: + service = req.CancelOperation.Service + operation = req.CancelOperation.Operation + default: + return nil, &nexuspb.HandlerError{ + ErrorType: string(nexus.HandlerErrorTypeNotImplemented), + Failure: &nexuspb.Failure{ + Message: "unknown request type", + }, + } + } + + return h.metricsHandler.WithTags(metrics.NexusTags(service, operation, h.taskQueueName)), nil +} + +func (h *nexusTaskHandler) fillInCompletion(taskToken []byte, res *nexuspb.Response) *workflowservice.RespondNexusTaskCompletedRequest { + return &workflowservice.RespondNexusTaskCompletedRequest{ + Identity: h.identity, + Namespace: h.namespace, + TaskToken: taskToken, + Response: res, + } +} + +func (h *nexusTaskHandler) fillInFailure(taskToken []byte, err *nexuspb.HandlerError) *workflowservice.RespondNexusTaskFailedRequest { + return &workflowservice.RespondNexusTaskFailedRequest{ + Identity: h.identity, + Namespace: h.namespace, + TaskToken: taskToken, + Error: err, + } +} + +// payloadSerializer is a fake nexus Serializer that uses a data converter to read from an embedded payload instead of +// using the given nexus.Context. Supports only Deserialize. +type payloadSerializer struct { + converter converter.DataConverter + payload *common.Payload +} + +func (p *payloadSerializer) Deserialize(_ *nexus.Content, v any) error { + return p.converter.FromPayload(p.payload, v) +} + +func (p *payloadSerializer) Serialize(v any) (*nexus.Content, error) { + panic("unimplemented") // not used - operation outputs are directly serialized to payload. +} + +var emptyReaderNopCloser = io.NopCloser(bytes.NewReader([]byte{})) + +// convertKnownErrors converts known errors to corresponding Nexus HandlerError. +func convertKnownErrors(err error) error { + // Handle common errors returned from various client methods. + if workflowErr, ok := err.(*WorkflowExecutionError); ok { + return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, workflowErr.Error()) + } + if queryRejectedErr, ok := err.(*QueryRejectedError); ok { + return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, queryRejectedErr.Error()) + } + + // Not using errors.As to be consistent ApplicationError checking with the rest of the SDK. + if appErr, ok := err.(*ApplicationError); ok { + if appErr.NonRetryable() { + return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, appErr.Error()) + } + return nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, appErr.Error()) + } + return convertServiceError(err) +} + +// convertServiceError converts a serviceerror into a Nexus HandlerError if possible. +// If exposeDetails is true, the error message from the given error is exposed in the converted HandlerError, otherwise, +// a default message with minimal information is attached to the returned error. +// Roughly taken from https://github.com/googleapis/googleapis/blob/master/google/rpc/code.proto +// and +// https://github.com/grpc-ecosystem/grpc-gateway/blob/a7cf811e6ffabeaddcfb4ff65602c12671ff326e/runtime/errors.go#L56. +func convertServiceError(err error) error { + var st *status.Status + + // Temporal serviceerrors have a Status() method. + stGetter, ok := err.(interface{ Status() *status.Status }) + if !ok { + // Not a serviceerror, passthrough. + return err + } + + st = stGetter.Status() + errMessage := err.Error() + + switch st.Code() { + case codes.AlreadyExists, codes.Canceled, codes.InvalidArgument, codes.FailedPrecondition, codes.OutOfRange: + return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, errMessage) + case codes.Aborted, codes.Unavailable: + return nexus.HandlerErrorf(nexus.HandlerErrorTypeUnavailable, errMessage) + case codes.DataLoss, codes.Internal, codes.Unknown, codes.Unauthenticated, codes.PermissionDenied: + // Note that codes.Unauthenticated, codes.PermissionDenied have Nexus error types but we convert to internal + // because this is not a client auth error and happens when the handler fails to auth with Temporal and should + // be considered retryable. + return nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, errMessage) + case codes.NotFound: + return nexus.HandlerErrorf(nexus.HandlerErrorTypeNotFound, errMessage) + case codes.ResourceExhausted: + return nexus.HandlerErrorf(nexus.HandlerErrorTypeResourceExhausted, errMessage) + case codes.Unimplemented: + return nexus.HandlerErrorf(nexus.HandlerErrorTypeNotImplemented, errMessage) + case codes.DeadlineExceeded: + return nexus.HandlerErrorf(nexus.HandlerErrorTypeDownstreamTimeout, errMessage) + } + + return err +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_nexus_task_poller.go b/vendor/go.temporal.io/sdk/internal/internal_nexus_task_poller.go new file mode 100644 index 00000000000..15a3d56f578 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_nexus_task_poller.go @@ -0,0 +1,193 @@ +// The MIT License +// +// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "time" + + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + taskqueuepb "go.temporal.io/api/taskqueue/v1" + "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/log" +) + +type nexusTaskPoller struct { + basePoller + namespace string + taskQueueName string + identity string + service workflowservice.WorkflowServiceClient + taskHandler *nexusTaskHandler + logger log.Logger + numPollerMetric *numPollerMetric +} + +type nexusTask struct { + task *workflowservice.PollNexusTaskQueueResponse +} + +var _ taskPoller = &nexusTaskPoller{} + +func newNexusTaskPoller( + taskHandler *nexusTaskHandler, + service workflowservice.WorkflowServiceClient, + params workerExecutionParameters, +) *nexusTaskPoller { + return &nexusTaskPoller{ + basePoller: basePoller{ + metricsHandler: params.MetricsHandler, + stopC: params.WorkerStopChannel, + workerBuildID: params.getBuildID(), + useBuildIDVersioning: params.UseBuildIDForVersioning, + capabilities: params.capabilities, + }, + taskHandler: taskHandler, + service: service, + namespace: params.Namespace, + taskQueueName: params.TaskQueue, + identity: params.Identity, + logger: params.Logger, + numPollerMetric: newNumPollerMetric(params.MetricsHandler, metrics.PollerTypeNexusTask), + } +} + +// Poll the nexus task queue and update the num_poller metric +func (ntp *nexusTaskPoller) pollNexusTaskQueue(ctx context.Context, request *workflowservice.PollNexusTaskQueueRequest) (*workflowservice.PollNexusTaskQueueResponse, error) { + ntp.numPollerMetric.increment() + defer ntp.numPollerMetric.decrement() + + return ntp.service.PollNexusTaskQueue(ctx, request) +} + +func (ntp *nexusTaskPoller) poll(ctx context.Context) (taskForWorker, error) { + traceLog(func() { + ntp.logger.Debug("nexusTaskPoller::Poll") + }) + request := &workflowservice.PollNexusTaskQueueRequest{ + Namespace: ntp.namespace, + TaskQueue: &taskqueuepb.TaskQueue{Name: ntp.taskQueueName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, + Identity: ntp.identity, + WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ + BuildId: ntp.workerBuildID, + UseVersioning: ntp.useBuildIDVersioning, + }, + } + + response, err := ntp.pollNexusTaskQueue(ctx, request) + if err != nil { + return nil, err + } + if response == nil || len(response.TaskToken) == 0 { + // No operation info is available on empty poll. Emit using base scope. + ntp.metricsHandler.Counter(metrics.NexusPollNoTaskCounter).Inc(1) + return nil, nil + } + + return &nexusTask{task: response}, nil +} + +// PollTask polls a new task +func (ntp *nexusTaskPoller) PollTask() (taskForWorker, error) { + return ntp.doPoll(ntp.poll) +} + +// ProcessTask processes a new task +func (ntp *nexusTaskPoller) ProcessTask(task interface{}) error { + if ntp.stopping() { + return errStop + } + + response := task.(*nexusTask).task + if response.GetRequest() == nil { + // We didn't get a request, poll must have timed out. + traceLog(func() { + ntp.logger.Debug("Empty Nexus poll response") + }) + return nil + } + + metricsHandler, handlerErr := ntp.taskHandler.metricsHandlerForTask(response) + if handlerErr != nil { + // context wasn't propagated to us, use a background context. + _, err := ntp.taskHandler.client.WorkflowService().RespondNexusTaskFailed( + context.Background(), ntp.taskHandler.fillInFailure(response.TaskToken, handlerErr)) + return err + } + + executionStartTime := time.Now() + + // Schedule-to-start (from the time the request hit the frontend). + scheduleToStartLatency := executionStartTime.Sub(response.GetRequest().GetScheduledTime().AsTime()) + metricsHandler.Timer(metrics.NexusTaskScheduleToStartLatency).Record(scheduleToStartLatency) + + // Process the nexus task. + res, failure, err := ntp.taskHandler.Execute(response) + + // Execution latency (in-SDK processing time). + metricsHandler.Timer(metrics.NexusTaskExecutionLatency).Record(time.Since(executionStartTime)) + if err != nil || failure != nil { + metricsHandler.Counter(metrics.NexusTaskExecutionFailedCounter).Inc(1) + } + + // Let the poller machinery drop the task, nothing to report back. + // This is only expected due to context deadline errors. + if err != nil { + return err + } + + if err := ntp.reportCompletion(res, failure); err != nil { + traceLog(func() { + ntp.logger.Debug("reportNexusTaskComplete failed", tagError, err) + }) + return err + } + + // E2E latency, from frontend until we finished reporting completion. + metricsHandler. + Timer(metrics.NexusTaskEndToEndLatency). + Record(time.Since(response.GetRequest().GetScheduledTime().AsTime())) + return nil +} + +func (ntp *nexusTaskPoller) reportCompletion( + completion *workflowservice.RespondNexusTaskCompletedRequest, + failure *workflowservice.RespondNexusTaskFailedRequest, +) error { + ctx := context.Background() + // No workflow or activity tags to report. + // Task queue expected to be empty for Respond*Task... requests. + rpcMetricsHandler := ntp.metricsHandler.WithTags(metrics.RPCTags(metrics.NoneTagValue, metrics.NoneTagValue, metrics.NoneTagValue)) + ctx, cancel := newGRPCContext(ctx, grpcMetricsHandler(rpcMetricsHandler), + defaultGrpcRetryParameters(ctx)) + defer cancel() + + if failure != nil { + _, err := ntp.taskHandler.client.WorkflowService().RespondNexusTaskFailed(ctx, failure) + return err + } + _, err := ntp.taskHandler.client.WorkflowService().RespondNexusTaskCompleted(ctx, completion) + return err +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_nexus_worker.go b/vendor/go.temporal.io/sdk/internal/internal_nexus_worker.go new file mode 100644 index 00000000000..3dbf424c1c6 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_nexus_worker.go @@ -0,0 +1,105 @@ +// The MIT License +// +// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "github.com/nexus-rpc/sdk-go/nexus" + "go.temporal.io/api/workflowservice/v1" +) + +type nexusWorkerOptions struct { + executionParameters workerExecutionParameters + client Client + workflowService workflowservice.WorkflowServiceClient + handler nexus.Handler +} + +type nexusWorker struct { + executionParameters workerExecutionParameters + workflowService workflowservice.WorkflowServiceClient + worker *baseWorker + stopC chan struct{} +} + +func newNexusWorker(opts nexusWorkerOptions) (*nexusWorker, error) { + workerStopChannel := make(chan struct{}) + params := opts.executionParameters + params.WorkerStopChannel = getReadOnlyChannel(workerStopChannel) + ensureRequiredParams(¶ms) + poller := newNexusTaskPoller( + newNexusTaskHandler( + opts.handler, + opts.executionParameters.Identity, + opts.executionParameters.Namespace, + opts.executionParameters.TaskQueue, + opts.client, + opts.executionParameters.DataConverter, + opts.executionParameters.Logger, + opts.executionParameters.MetricsHandler, + ), + opts.workflowService, + params, + ) + + baseWorker := newBaseWorker(baseWorkerOptions{ + pollerCount: params.MaxConcurrentNexusTaskQueuePollers, + pollerRate: defaultPollerRate, + slotSupplier: params.Tuner.GetNexusSlotSupplier(), + maxTaskPerSecond: defaultWorkerTaskExecutionRate, + taskWorker: poller, + workerType: "NexusWorker", + identity: params.Identity, + buildId: params.getBuildID(), + logger: params.Logger, + stopTimeout: params.WorkerStopTimeout, + fatalErrCb: params.WorkerFatalErrorCallback, + metricsHandler: params.MetricsHandler, + slotReservationData: slotReservationData{ + taskQueue: params.TaskQueue, + }, + }, + ) + + return &nexusWorker{ + executionParameters: opts.executionParameters, + workflowService: opts.workflowService, + worker: baseWorker, + stopC: workerStopChannel, + }, nil +} + +// Start the worker. +func (w *nexusWorker) Start() error { + err := verifyNamespaceExist(w.workflowService, w.executionParameters.MetricsHandler, w.executionParameters.Namespace, w.worker.logger) + if err != nil { + return err + } + w.worker.Start() + return nil +} + +// Stop the worker. +func (w *nexusWorker) Stop() { + close(w.stopC) + w.worker.Stop() +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_pressure_points.go b/vendor/go.temporal.io/sdk/internal/internal_pressure_points.go new file mode 100644 index 00000000000..0849f24656e --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_pressure_points.go @@ -0,0 +1,95 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "errors" + "math/rand" + "strconv" + "time" + + "go.temporal.io/api/workflowservice/v1" + + "go.temporal.io/sdk/log" +) + +// ** This is for internal stress testing framework ** + +// PressurePoints +const ( + pressurePointTypeWorkflowTaskStartTimeout = "workflow-task-start-timeout" + pressurePointTypeWorkflowTaskCompleted = "workflow-task-complete" + pressurePointTypeActivityTaskScheduleTimeout = "activity-task-schedule-timeout" + pressurePointTypeActivityTaskStartTimeout = "activity-task-start-timeout" + pressurePointConfigProbability = "probability" + pressurePointConfigSleep = "sleep" + workerOptionsConfig = "worker-options" + workerOptionsConfigConcurrentPollRoutineSize = "ConcurrentPollRoutineSize" +) + +type ( + pressurePointMgr interface { + Execute(pressurePointName string) error + } + + pressurePointMgrImpl struct { + config map[string]map[string]string + logger log.Logger + } +) + +// newWorkflowWorkerWithPressurePoints returns an instance of a workflow worker. +func newWorkflowWorkerWithPressurePoints(service workflowservice.WorkflowServiceClient, params workerExecutionParameters, pressurePoints map[string]map[string]string, registry *registry) (worker *workflowWorker) { + return newWorkflowWorker(service, params, &pressurePointMgrImpl{config: pressurePoints, logger: params.Logger}, registry) +} + +func (p *pressurePointMgrImpl) Execute(pressurePointName string) error { + if config, ok := p.config[pressurePointName]; ok { + // If probability is configured. + if value, ok2 := config[pressurePointConfigProbability]; ok2 { + if probability, err := strconv.Atoi(value); err == nil { + if rand.Int31n(100) < int32(probability) { + // Drop the task. + p.logger.Debug("pressurePointMgrImpl.Execute drop task.", + "PressurePointName", pressurePointName, + "probability", probability) + return errors.New("pressurepoint configured") + } + } + } else if value, ok3 := config[pressurePointConfigSleep]; ok3 { + if timeoutSeconds, err := strconv.Atoi(value); err == nil { + if timeoutSeconds > 0 { + p.logger.Debug("pressurePointMgrImpl.Execute sleep.", + "PressurePointName", pressurePointName, + "DurationSeconds", timeoutSeconds) + d := time.Duration(timeoutSeconds) * time.Second + time.Sleep(d) + return nil + } + } + } + } + return nil +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_public.go b/vendor/go.temporal.io/sdk/internal/internal_public.go new file mode 100644 index 00000000000..baf982d242b --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_public.go @@ -0,0 +1,123 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +// WARNING! WARNING! WARNING! WARNING! WARNING! WARNING! WARNING! WARNING! WARNING! +// Any of the APIs in this file are not supported for application level developers +// and are subject to change without any notice. +// +// APIs that are internal to Temporal system developers and are public from the Go +// point of view only to access them from other packages. + +import ( + "time" + + historypb "go.temporal.io/api/history/v1" + "go.temporal.io/api/workflowservice/v1" +) + +type ( + workflowTaskHeartbeatFunc func(response interface{}, startTime time.Time) (*workflowTask, error) + + // HistoryIterator iterator through history events + HistoryIterator interface { + // GetNextPage returns next page of history events + GetNextPage() (*historypb.History, error) + // Reset resets the internal state so next GetNextPage() call will return first page of events from beginning. + Reset() + // HasNextPage returns if there are more page of events + HasNextPage() bool + } + + // WorkflowExecutionContext represents one instance of workflow execution state in memory. Lock must be obtained before + // calling into any methods. + WorkflowExecutionContext interface { + Lock() + Unlock(err error) + ProcessWorkflowTask(workflowTask *workflowTask) (completeRequest interface{}, err error) + ProcessLocalActivityResult(workflowTask *workflowTask, lar *localActivityResult) (interface{}, error) + // CompleteWorkflowTask try to complete current workflow task and get response that needs to be sent back to server. + // The waitLocalActivity is used to control if we should wait for outstanding local activities. + // If there is no outstanding local activities or if waitLocalActivity is false, the complete will return response + // which will be one of following: + // - RespondWorkflowTaskCompletedRequest + // - RespondWorkflowTaskFailedRequest + // - RespondQueryTaskCompletedRequest + // If waitLocalActivity is true, and there is outstanding local activities, this call will return nil. + CompleteWorkflowTask(workflowTask *workflowTask, waitLocalActivity bool) interface{} + // GetWorkflowTaskTimeout returns the WorkflowTaskTimeout + GetWorkflowTaskTimeout() time.Duration + GetCurrentWorkflowTask() *workflowservice.PollWorkflowTaskQueueResponse + IsDestroyed() bool + StackTrace() string + } + + EventLevelResetter func(int64) + + // WorkflowTaskHandler represents workflow task handlers. + WorkflowTaskHandler interface { + WorkflowContextManager + + // Processes the workflow task + // The response could be: + // - RespondWorkflowTaskCompletedRequest + // - RespondWorkflowTaskFailedRequest + // - RespondQueryTaskCompletedRequest + ProcessWorkflowTask( + task *workflowTask, + ctx *workflowExecutionContextImpl, + f workflowTaskHeartbeatFunc, + ) (response interface{}, err error) + } + + WorkflowContextManager interface { + // GetOrCreateWorkflowContext finds an existing cached context object + // for the provided task's run ID or creates a new object, adds it to + // cache, and returns it. In all non-error cases the returned context + // object is in a locked state (i.e. + // workflowExecutionContextImpl.Lock() has been called). + GetOrCreateWorkflowContext( + task *workflowservice.PollWorkflowTaskQueueResponse, + historyIterator HistoryIterator, + ) (*workflowExecutionContextImpl, error) + } + + // ActivityTaskHandler represents activity task handlers. + ActivityTaskHandler interface { + // Executes the activity task + // The response is one of the types: + // - RespondActivityTaskCompletedRequest + // - RespondActivityTaskFailedRequest + // - RespondActivityTaskCanceledRequest + Execute(taskQueue string, task *workflowservice.PollActivityTaskQueueResponse) (interface{}, error) + } +) + +var enableVerboseLogging = false + +// EnableVerboseLogging enable or disable verbose logging. This is for internal use only. +func EnableVerboseLogging(enable bool) { + enableVerboseLogging = enable +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_retry.go b/vendor/go.temporal.io/sdk/internal/internal_retry.go new file mode 100644 index 00000000000..933afab0f5c --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_retry.go @@ -0,0 +1,65 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +// All code in this file is private to the package. + +import ( + "context" + "time" + + "go.temporal.io/sdk/internal/common/backoff" +) + +const ( + retryServiceOperationInitialInterval = 200 * time.Millisecond + retryServiceOperationExpirationInterval = 60 * time.Second + retryServiceOperationBackoff = 2 +) + +// Creates a retry policy which allows appropriate retries for the deadline passed in as context. +// It uses the context deadline to set MaxInterval as 1/10th of context timeout +// MaxInterval = Max(context_timeout/10, 20ms) +// defaults to ExpirationInterval of 60 seconds, or uses context deadline as expiration interval +func createDynamicServiceRetryPolicy(ctx context.Context) backoff.RetryPolicy { + timeout := retryServiceOperationExpirationInterval + if ctx != nil { + now := time.Now() + if expiration, ok := ctx.Deadline(); ok && expiration.After(now) { + timeout = expiration.Sub(now) + } + } + initialInterval := retryServiceOperationInitialInterval + maximumInterval := timeout / 10 + if maximumInterval < retryServiceOperationInitialInterval { + maximumInterval = retryServiceOperationInitialInterval + } + + policy := backoff.NewExponentialRetryPolicy(initialInterval) + policy.SetBackoffCoefficient(retryServiceOperationBackoff) + policy.SetMaximumInterval(maximumInterval) + policy.SetExpirationInterval(timeout) + return policy +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_schedule_client.go b/vendor/go.temporal.io/sdk/internal/internal_schedule_client.go new file mode 100644 index 00000000000..5de1df4c341 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_schedule_client.go @@ -0,0 +1,876 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "errors" + "fmt" + "time" + + "google.golang.org/protobuf/types/known/durationpb" + "google.golang.org/protobuf/types/known/timestamppb" + + "github.com/pborman/uuid" + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + schedulepb "go.temporal.io/api/schedule/v1" + "go.temporal.io/api/serviceerror" + taskqueuepb "go.temporal.io/api/taskqueue/v1" + workflowpb "go.temporal.io/api/workflow/v1" + "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/log" +) + +type ( + + // ScheduleClient is the client for starting a workflow execution. + scheduleClient struct { + workflowClient *WorkflowClient + } + + // scheduleHandleImpl is the implementation of ScheduleHandle. + scheduleHandleImpl struct { + ID string + client *WorkflowClient + } + + // scheduleListIteratorImpl is the implementation of ScheduleListIterator + scheduleListIteratorImpl struct { + // nextScheduleIndex - Local cached schedules events and corresponding consuming index + nextScheduleIndex int + + // err - From getting the latest page of schedules + err error + + // response - From getting the latest page of schedules + response *workflowservice.ListSchedulesResponse + + // paginate - Function which use a next token to get next page of schedules events + paginate func(nexttoken []byte) (*workflowservice.ListSchedulesResponse, error) + } +) + +func (w *workflowClientInterceptor) CreateSchedule(ctx context.Context, in *ScheduleClientCreateInput) (ScheduleHandle, error) { + // This is always set before interceptor is invoked + ID := in.Options.ID + if ID == "" { + return nil, fmt.Errorf("no schedule ID in options") + } + + dataConverter := WithContext(ctx, w.client.dataConverter) + if dataConverter == nil { + dataConverter = converter.GetDefaultDataConverter() + } + + if in.Options.Action == nil { + return nil, fmt.Errorf("no schedule action in options") + } + action, err := convertToPBScheduleAction(ctx, w.client, in.Options.Action) + if err != nil { + return nil, err + } + + memo, err := getWorkflowMemo(in.Options.Memo, dataConverter) + if err != nil { + return nil, err + } + + searchAttr, err := serializeSearchAttributes(in.Options.SearchAttributes, in.Options.TypedSearchAttributes) + if err != nil { + return nil, err + } + + var triggerImmediately *schedulepb.TriggerImmediatelyRequest + if in.Options.TriggerImmediately { + triggerImmediately = &schedulepb.TriggerImmediatelyRequest{ + OverlapPolicy: in.Options.Overlap, + } + } + + backfillRequests := convertToPBBackfillList(in.Options.ScheduleBackfill) + + // Only send an initial patch if we need to. + var initialPatch *schedulepb.SchedulePatch + if in.Options.TriggerImmediately || len(in.Options.ScheduleBackfill) > 0 { + initialPatch = &schedulepb.SchedulePatch{ + TriggerImmediately: triggerImmediately, + BackfillRequest: backfillRequests, + } + } + + var catchupWindow *durationpb.Duration + if in.Options.CatchupWindow != 0 { + // Convert to nil so the server uses the default + // catchup window,otherwise it will use the minimum (10s). + catchupWindow = durationpb.New(in.Options.CatchupWindow) + } + + // run propagators to extract information about tracing and other stuff, store in headers field + startRequest := &workflowservice.CreateScheduleRequest{ + Namespace: w.client.namespace, + ScheduleId: ID, + RequestId: uuid.New(), + Schedule: &schedulepb.Schedule{ + Spec: convertToPBScheduleSpec(&in.Options.Spec), + Action: action, + Policies: &schedulepb.SchedulePolicies{ + OverlapPolicy: in.Options.Overlap, + CatchupWindow: catchupWindow, + PauseOnFailure: in.Options.PauseOnFailure, + }, + State: &schedulepb.ScheduleState{ + Notes: in.Options.Note, + Paused: in.Options.Paused, + LimitedActions: in.Options.RemainingActions != 0, + RemainingActions: int64(in.Options.RemainingActions), + }, + }, + InitialPatch: initialPatch, + Identity: w.client.identity, + Memo: memo, + SearchAttributes: searchAttr, + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + _, err = w.client.workflowService.CreateSchedule(grpcCtx, startRequest) + if _, ok := err.(*serviceerror.WorkflowExecutionAlreadyStarted); ok { + return nil, ErrScheduleAlreadyRunning + } + if err != nil { + return nil, err + } + + return &scheduleHandleImpl{ + ID: ID, + client: w.client, + }, nil +} + +func (sc *scheduleClient) Create(ctx context.Context, options ScheduleOptions) (ScheduleHandle, error) { + if err := sc.workflowClient.ensureInitialized(ctx); err != nil { + return nil, err + } + + // Set header before interceptor run + ctx = contextWithNewHeader(ctx) + + // Run via interceptor + return sc.workflowClient.interceptor.CreateSchedule(ctx, &ScheduleClientCreateInput{ + Options: &options, + }) +} + +func (sc *scheduleClient) GetHandle(ctx context.Context, scheduleID string) ScheduleHandle { + return &scheduleHandleImpl{ + ID: scheduleID, + client: sc.workflowClient, + } +} + +func (sc *scheduleClient) List(ctx context.Context, options ScheduleListOptions) (ScheduleListIterator, error) { + paginate := func(nextToken []byte) (*workflowservice.ListSchedulesResponse, error) { + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + request := &workflowservice.ListSchedulesRequest{ + Namespace: sc.workflowClient.namespace, + MaximumPageSize: int32(options.PageSize), + NextPageToken: nextToken, + Query: options.Query, + } + + return sc.workflowClient.workflowService.ListSchedules(grpcCtx, request) + } + + return &scheduleListIteratorImpl{ + paginate: paginate, + }, nil +} + +func (iter *scheduleListIteratorImpl) HasNext() bool { + if iter.err == nil { + if iter.response == nil || + (iter.nextScheduleIndex >= len(iter.response.Schedules) && len(iter.response.NextPageToken) > 0) { + iter.response, iter.err = iter.paginate(iter.response.GetNextPageToken()) + iter.nextScheduleIndex = 0 + } + } + return iter.nextScheduleIndex < len(iter.response.GetSchedules()) || iter.err != nil +} + +func (iter *scheduleListIteratorImpl) Next() (*ScheduleListEntry, error) { + if !iter.HasNext() { + panic("ScheduleListIterator Next() called without checking HasNext()") + } else if iter.err != nil { + return nil, iter.err + } + schedule := iter.response.Schedules[iter.nextScheduleIndex] + iter.nextScheduleIndex++ + return convertFromPBScheduleListEntry(schedule), nil +} + +func (scheduleHandle *scheduleHandleImpl) GetID() string { + return scheduleHandle.ID +} + +func (scheduleHandle *scheduleHandleImpl) Delete(ctx context.Context) error { + request := &workflowservice.DeleteScheduleRequest{ + Namespace: scheduleHandle.client.namespace, + ScheduleId: scheduleHandle.ID, + Identity: scheduleHandle.client.identity, + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := scheduleHandle.client.workflowService.DeleteSchedule(grpcCtx, request) + return err +} + +func (scheduleHandle *scheduleHandleImpl) Backfill(ctx context.Context, options ScheduleBackfillOptions) error { + request := &workflowservice.PatchScheduleRequest{ + Namespace: scheduleHandle.client.namespace, + ScheduleId: scheduleHandle.ID, + Patch: &schedulepb.SchedulePatch{ + BackfillRequest: convertToPBBackfillList(options.Backfill), + }, + Identity: scheduleHandle.client.identity, + RequestId: uuid.New(), + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := scheduleHandle.client.workflowService.PatchSchedule(grpcCtx, request) + return err +} + +func (scheduleHandle *scheduleHandleImpl) Update(ctx context.Context, options ScheduleUpdateOptions) error { + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + ctx = contextWithNewHeader(ctx) + + describeRequest := &workflowservice.DescribeScheduleRequest{ + Namespace: scheduleHandle.client.namespace, + ScheduleId: scheduleHandle.ID, + } + describeResponse, err := scheduleHandle.client.workflowService.DescribeSchedule(grpcCtx, describeRequest) + if err != nil { + return err + } + scheduleDescription, err := scheduleDescriptionFromPB(scheduleHandle.client.logger, describeResponse) + if err != nil { + return err + } + newSchedule, err := options.DoUpdate(ScheduleUpdateInput{ + Description: *scheduleDescription, + }) + if err != nil { + if errors.Is(err, ErrSkipScheduleUpdate) { + return nil + } + return err + } + newSchedulePB, err := convertToPBSchedule(ctx, scheduleHandle.client, newSchedule.Schedule) + if err != nil { + return err + } + + var newSA *commonpb.SearchAttributes + attributes := newSchedule.TypedSearchAttributes + if attributes != nil { + newSA, err = serializeTypedSearchAttributes(attributes.GetUntypedValues()) + if err != nil { + return err + } + } + + _, err = scheduleHandle.client.workflowService.UpdateSchedule(grpcCtx, &workflowservice.UpdateScheduleRequest{ + Namespace: scheduleHandle.client.namespace, + ScheduleId: scheduleHandle.ID, + Schedule: newSchedulePB, + ConflictToken: nil, + Identity: scheduleHandle.client.identity, + RequestId: uuid.New(), + SearchAttributes: newSA, + }) + return err +} + +func (scheduleHandle *scheduleHandleImpl) Describe(ctx context.Context) (*ScheduleDescription, error) { + request := &workflowservice.DescribeScheduleRequest{ + Namespace: scheduleHandle.client.namespace, + ScheduleId: scheduleHandle.ID, + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + describeResponse, err := scheduleHandle.client.workflowService.DescribeSchedule(grpcCtx, request) + if err != nil { + return nil, err + } + return scheduleDescriptionFromPB(scheduleHandle.client.logger, describeResponse) +} + +func (scheduleHandle *scheduleHandleImpl) Trigger(ctx context.Context, options ScheduleTriggerOptions) error { + request := &workflowservice.PatchScheduleRequest{ + Namespace: scheduleHandle.client.namespace, + ScheduleId: scheduleHandle.ID, + Patch: &schedulepb.SchedulePatch{ + TriggerImmediately: &schedulepb.TriggerImmediatelyRequest{ + OverlapPolicy: options.Overlap, + }, + }, + Identity: scheduleHandle.client.identity, + RequestId: uuid.New(), + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := scheduleHandle.client.workflowService.PatchSchedule(grpcCtx, request) + return err +} + +func (scheduleHandle *scheduleHandleImpl) Pause(ctx context.Context, options SchedulePauseOptions) error { + pauseNote := "Paused via Go SDK" + if options.Note != "" { + pauseNote = options.Note + } + request := &workflowservice.PatchScheduleRequest{ + Namespace: scheduleHandle.client.namespace, + ScheduleId: scheduleHandle.ID, + Patch: &schedulepb.SchedulePatch{ + Pause: pauseNote, + }, + Identity: scheduleHandle.client.identity, + RequestId: uuid.New(), + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := scheduleHandle.client.workflowService.PatchSchedule(grpcCtx, request) + return err +} + +func (scheduleHandle *scheduleHandleImpl) Unpause(ctx context.Context, options ScheduleUnpauseOptions) error { + unpauseNote := "Unpaused via Go SDK" + if options.Note != "" { + unpauseNote = options.Note + } + request := &workflowservice.PatchScheduleRequest{ + Namespace: scheduleHandle.client.namespace, + ScheduleId: scheduleHandle.ID, + Patch: &schedulepb.SchedulePatch{ + Unpause: unpauseNote, + }, + Identity: scheduleHandle.client.identity, + RequestId: uuid.New(), + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := scheduleHandle.client.workflowService.PatchSchedule(grpcCtx, request) + return err +} + +func convertToPBScheduleSpec(scheduleSpec *ScheduleSpec) *schedulepb.ScheduleSpec { + if scheduleSpec == nil { + return nil + } + + calendar := convertToPBScheduleCalendarSpecList(scheduleSpec.Calendars) + + intervals := make([]*schedulepb.IntervalSpec, len(scheduleSpec.Intervals)) + for i, interval := range scheduleSpec.Intervals { + intervalSpec := interval + intervals[i] = &schedulepb.IntervalSpec{ + Interval: durationpb.New(intervalSpec.Every), + Phase: durationpb.New(intervalSpec.Offset), + } + } + + skip := convertToPBScheduleCalendarSpecList(scheduleSpec.Skip) + + var startTime *timestamppb.Timestamp + if !scheduleSpec.StartAt.IsZero() { + startTime = timestamppb.New(scheduleSpec.StartAt) + } + + var endTime *timestamppb.Timestamp + if !scheduleSpec.EndAt.IsZero() { + endTime = timestamppb.New(scheduleSpec.EndAt) + } + + return &schedulepb.ScheduleSpec{ + StructuredCalendar: calendar, + Interval: intervals, + CronString: scheduleSpec.CronExpressions, + ExcludeStructuredCalendar: skip, + StartTime: startTime, + EndTime: endTime, + Jitter: durationpb.New(scheduleSpec.Jitter), + // TODO support custom time zone data + TimezoneName: scheduleSpec.TimeZoneName, + } +} + +func convertFromPBScheduleSpec(scheduleSpec *schedulepb.ScheduleSpec) *ScheduleSpec { + if scheduleSpec == nil { + return nil + } + + calendars := convertFromPBScheduleCalendarSpecList(scheduleSpec.GetStructuredCalendar()) + + intervals := make([]ScheduleIntervalSpec, len(scheduleSpec.GetInterval())) + for i, s := range scheduleSpec.GetInterval() { + intervals[i] = ScheduleIntervalSpec{ + Every: s.Interval.AsDuration(), + Offset: s.Phase.AsDuration(), + } + } + + skip := convertFromPBScheduleCalendarSpecList(scheduleSpec.GetExcludeStructuredCalendar()) + + startAt := time.Time{} + if scheduleSpec.GetStartTime() != nil { + startAt = scheduleSpec.GetStartTime().AsTime() + } + + endAt := time.Time{} + if scheduleSpec.GetEndTime() != nil { + endAt = scheduleSpec.GetEndTime().AsTime() + } + + return &ScheduleSpec{ + Calendars: calendars, + Intervals: intervals, + Skip: skip, + StartAt: startAt, + EndAt: endAt, + Jitter: scheduleSpec.GetJitter().AsDuration(), + TimeZoneName: scheduleSpec.GetTimezoneName(), + } +} + +func scheduleDescriptionFromPB( + logger log.Logger, + describeResponse *workflowservice.DescribeScheduleResponse, +) (*ScheduleDescription, error) { + if describeResponse == nil { + return nil, nil + } + + runningWorkflows := make([]ScheduleWorkflowExecution, len(describeResponse.Info.GetRunningWorkflows())) + for i, s := range describeResponse.Info.GetRunningWorkflows() { + runningWorkflows[i] = ScheduleWorkflowExecution{ + WorkflowID: s.GetWorkflowId(), + FirstExecutionRunID: s.GetRunId(), + } + } + + recentActions := convertFromPBScheduleActionResultList(describeResponse.Info.GetRecentActions()) + + nextActionTimes := make([]time.Time, len(describeResponse.Info.GetFutureActionTimes())) + for i, t := range describeResponse.Info.GetFutureActionTimes() { + nextActionTimes[i] = t.AsTime() + } + + actionDescription, err := convertFromPBScheduleAction(logger, describeResponse.Schedule.Action) + if err != nil { + return nil, err + } + + var typedSearchAttributes SearchAttributes + searchAttributes := describeResponse.SearchAttributes + if searchAttributes != nil { + typedSearchAttributes = convertToTypedSearchAttributes(logger, searchAttributes.IndexedFields) + } + + return &ScheduleDescription{ + Schedule: Schedule{ + Action: actionDescription, + Spec: convertFromPBScheduleSpec(describeResponse.Schedule.Spec), + Policy: &SchedulePolicies{ + Overlap: describeResponse.Schedule.Policies.GetOverlapPolicy(), + CatchupWindow: describeResponse.Schedule.Policies.GetCatchupWindow().AsDuration(), + PauseOnFailure: describeResponse.Schedule.Policies.GetPauseOnFailure(), + }, + State: &ScheduleState{ + Note: describeResponse.Schedule.State.GetNotes(), + Paused: describeResponse.Schedule.State.GetPaused(), + LimitedActions: describeResponse.Schedule.State.GetLimitedActions(), + RemainingActions: int(describeResponse.Schedule.State.GetRemainingActions()), + }, + }, + Info: ScheduleInfo{ + NumActions: int(describeResponse.Info.ActionCount), + NumActionsMissedCatchupWindow: int(describeResponse.Info.MissedCatchupWindow), + NumActionsSkippedOverlap: int(describeResponse.Info.OverlapSkipped), + RunningWorkflows: runningWorkflows, + RecentActions: recentActions, + NextActionTimes: nextActionTimes, + CreatedAt: describeResponse.Info.GetCreateTime().AsTime(), + LastUpdateAt: describeResponse.Info.GetUpdateTime().AsTime(), + }, + Memo: describeResponse.Memo, + SearchAttributes: searchAttributes, + TypedSearchAttributes: typedSearchAttributes, + }, nil +} + +func convertToPBSchedule(ctx context.Context, client *WorkflowClient, schedule *Schedule) (*schedulepb.Schedule, error) { + if schedule == nil { + return nil, nil + } + action, err := convertToPBScheduleAction(ctx, client, schedule.Action) + if err != nil { + return nil, err + } + return &schedulepb.Schedule{ + Spec: convertToPBScheduleSpec(schedule.Spec), + Action: action, + Policies: &schedulepb.SchedulePolicies{ + OverlapPolicy: schedule.Policy.Overlap, + CatchupWindow: durationpb.New(schedule.Policy.CatchupWindow), + PauseOnFailure: schedule.Policy.PauseOnFailure, + }, + State: &schedulepb.ScheduleState{ + Notes: schedule.State.Note, + Paused: schedule.State.Paused, + LimitedActions: schedule.State.LimitedActions, + RemainingActions: int64(schedule.State.RemainingActions), + }, + }, nil +} + +func convertFromPBScheduleListEntry(schedule *schedulepb.ScheduleListEntry) *ScheduleListEntry { + scheduleInfo := schedule.GetInfo() + + recentActions := convertFromPBScheduleActionResultList(scheduleInfo.GetRecentActions()) + + nextActionTimes := make([]time.Time, len(schedule.Info.GetFutureActionTimes())) + for i, t := range schedule.Info.GetFutureActionTimes() { + nextActionTimes[i] = t.AsTime() + } + + return &ScheduleListEntry{ + ID: schedule.ScheduleId, + Spec: convertFromPBScheduleSpec(scheduleInfo.GetSpec()), + Note: scheduleInfo.GetNotes(), + Paused: scheduleInfo.GetPaused(), + WorkflowType: WorkflowType{ + Name: scheduleInfo.GetWorkflowType().GetName(), + }, + RecentActions: recentActions, + NextActionTimes: nextActionTimes, + Memo: schedule.Memo, + SearchAttributes: schedule.SearchAttributes, + } +} + +func convertToPBScheduleAction( + ctx context.Context, + client *WorkflowClient, + scheduleAction ScheduleAction, +) (*schedulepb.ScheduleAction, error) { + switch action := scheduleAction.(type) { + case *ScheduleWorkflowAction: + // Set header before interceptor run + dataConverter := WithContext(ctx, client.dataConverter) + + // Default workflow ID + if action.ID == "" { + action.ID = uuid.New() + } + + // Validate function and get name + if err := validateFunctionArgs(action.Workflow, action.Args, true); err != nil { + return nil, err + } + workflowType, err := getWorkflowFunctionName(client.registry, action.Workflow) + if err != nil { + return nil, err + } + // Encode workflow inputs that may already be encoded + input, err := encodeScheduleWorklowArgs(dataConverter, action.Args) + if err != nil { + return nil, err + } + // Encode workflow memos that may already be encoded + memo, err := encodeScheduleWorkflowMemo(dataConverter, action.Memo) + if err != nil { + return nil, err + } + + searchAttrs, err := serializeSearchAttributes(nil, action.TypedSearchAttributes) + if err != nil { + return nil, err + } + // Add any untyped search attributes that aren't already there + for k, v := range action.UntypedSearchAttributes { + if searchAttrs.GetIndexedFields()[k] == nil { + if searchAttrs == nil || searchAttrs.IndexedFields == nil { + searchAttrs = &commonpb.SearchAttributes{IndexedFields: map[string]*commonpb.Payload{}} + } + searchAttrs.IndexedFields[k] = v + } + } + + // get workflow headers from the context + header, err := headerPropagated(ctx, client.contextPropagators) + if err != nil { + return nil, err + } + + userMetadata, err := buildUserMetadata(action.staticSummary, action.staticDetails, dataConverter) + if err != nil { + return nil, err + } + + return &schedulepb.ScheduleAction{ + Action: &schedulepb.ScheduleAction_StartWorkflow{ + StartWorkflow: &workflowpb.NewWorkflowExecutionInfo{ + WorkflowId: action.ID, + WorkflowType: &commonpb.WorkflowType{Name: workflowType}, + TaskQueue: &taskqueuepb.TaskQueue{Name: action.TaskQueue, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, + Input: input, + WorkflowExecutionTimeout: durationpb.New(action.WorkflowExecutionTimeout), + WorkflowRunTimeout: durationpb.New(action.WorkflowRunTimeout), + WorkflowTaskTimeout: durationpb.New(action.WorkflowTaskTimeout), + RetryPolicy: convertToPBRetryPolicy(action.RetryPolicy), + Memo: memo, + SearchAttributes: searchAttrs, + Header: header, + UserMetadata: userMetadata, + }, + }, + }, nil + default: + // TODO maybe just panic instead? + return nil, fmt.Errorf("could not parse ScheduleAction") + } +} + +func convertFromPBScheduleAction(logger log.Logger, action *schedulepb.ScheduleAction) (ScheduleAction, error) { + switch action := action.Action.(type) { + case *schedulepb.ScheduleAction_StartWorkflow: + workflow := action.StartWorkflow + + args := make([]interface{}, len(workflow.GetInput().GetPayloads())) + for i, p := range workflow.GetInput().GetPayloads() { + args[i] = p + } + + memos := make(map[string]interface{}) + for key, element := range workflow.GetMemo().GetFields() { + memos[key] = element + } + + searchAttrs := convertToTypedSearchAttributes(logger, workflow.GetSearchAttributes().GetIndexedFields()) + // Create untyped list for any attribute not in the existing list + untypedSearchAttrs := map[string]*commonpb.Payload{} + for k, v := range workflow.GetSearchAttributes().GetIndexedFields() { + var inTyped bool + for typedKey := range searchAttrs.untypedValue { + if inTyped = typedKey.GetName() == k; inTyped { + break + } + } + if !inTyped { + untypedSearchAttrs[k] = v + } + } + + return &ScheduleWorkflowAction{ + ID: workflow.GetWorkflowId(), + Workflow: workflow.WorkflowType.GetName(), + Args: args, + TaskQueue: workflow.TaskQueue.GetName(), + WorkflowExecutionTimeout: workflow.GetWorkflowExecutionTimeout().AsDuration(), + WorkflowRunTimeout: workflow.GetWorkflowRunTimeout().AsDuration(), + WorkflowTaskTimeout: workflow.GetWorkflowTaskTimeout().AsDuration(), + RetryPolicy: convertFromPBRetryPolicy(workflow.RetryPolicy), + Memo: memos, + TypedSearchAttributes: searchAttrs, + UntypedSearchAttributes: untypedSearchAttrs, + }, nil + default: + // TODO maybe just panic instead? + return nil, fmt.Errorf("could not parse ScheduleAction") + } +} + +func convertToPBBackfillList(backfillRequests []ScheduleBackfill) []*schedulepb.BackfillRequest { + backfillRequestsPB := make([]*schedulepb.BackfillRequest, len(backfillRequests)) + for i, b := range backfillRequests { + backfill := b + backfillRequestsPB[i] = &schedulepb.BackfillRequest{ + StartTime: timestamppb.New(backfill.Start), + EndTime: timestamppb.New(backfill.End), + OverlapPolicy: backfill.Overlap, + } + } + return backfillRequestsPB +} + +func convertToPBRangeList(scheduleRange []ScheduleRange) []*schedulepb.Range { + rangesPB := make([]*schedulepb.Range, len(scheduleRange)) + for i, r := range scheduleRange { + rangesPB[i] = &schedulepb.Range{ + Start: int32(r.Start), + End: int32(r.End), + Step: int32(r.Step), + } + } + return rangesPB +} + +func convertFromPBRangeList(scheduleRangePB []*schedulepb.Range) []ScheduleRange { + scheduleRange := make([]ScheduleRange, len(scheduleRangePB)) + for i, r := range scheduleRangePB { + if r == nil { + continue + } + scheduleRange[i] = ScheduleRange{ + Start: int(r.Start), + End: int(r.End), + Step: int(r.Step), + } + } + return scheduleRange +} + +func convertFromPBScheduleCalendarSpecList(calendarSpecPB []*schedulepb.StructuredCalendarSpec) []ScheduleCalendarSpec { + calendarSpec := make([]ScheduleCalendarSpec, len(calendarSpecPB)) + for i, e := range calendarSpecPB { + calendarSpec[i] = ScheduleCalendarSpec{ + Second: convertFromPBRangeList(e.Second), + Minute: convertFromPBRangeList(e.Minute), + Hour: convertFromPBRangeList(e.Hour), + DayOfMonth: convertFromPBRangeList(e.DayOfMonth), + Month: convertFromPBRangeList(e.Month), + Year: convertFromPBRangeList(e.Year), + DayOfWeek: convertFromPBRangeList(e.DayOfWeek), + Comment: e.Comment, + } + } + return calendarSpec +} + +func applyScheduleCalendarSpecDefault(calendarSpec *ScheduleCalendarSpec) { + if calendarSpec.Second == nil { + calendarSpec.Second = []ScheduleRange{{Start: 0}} + } + + if calendarSpec.Minute == nil { + calendarSpec.Minute = []ScheduleRange{{Start: 0}} + } + + if calendarSpec.Hour == nil { + calendarSpec.Hour = []ScheduleRange{{Start: 0}} + } + + if calendarSpec.DayOfMonth == nil { + calendarSpec.DayOfMonth = []ScheduleRange{{Start: 1, End: 31}} + } + + if calendarSpec.Month == nil { + calendarSpec.Month = []ScheduleRange{{Start: 1, End: 12}} + } + + if calendarSpec.DayOfWeek == nil { + calendarSpec.DayOfWeek = []ScheduleRange{{Start: 0, End: 6}} + } +} + +func convertToPBScheduleCalendarSpecList(calendarSpec []ScheduleCalendarSpec) []*schedulepb.StructuredCalendarSpec { + calendarSpecPB := make([]*schedulepb.StructuredCalendarSpec, len(calendarSpec)) + for i, e := range calendarSpec { + applyScheduleCalendarSpecDefault(&e) + + calendarSpecPB[i] = &schedulepb.StructuredCalendarSpec{ + Second: convertToPBRangeList(e.Second), + Minute: convertToPBRangeList(e.Minute), + Hour: convertToPBRangeList(e.Hour), + DayOfMonth: convertToPBRangeList(e.DayOfMonth), + Month: convertToPBRangeList(e.Month), + Year: convertToPBRangeList(e.Year), + DayOfWeek: convertToPBRangeList(e.DayOfWeek), + Comment: e.Comment, + } + } + return calendarSpecPB +} + +func convertFromPBScheduleActionResultList(aa []*schedulepb.ScheduleActionResult) []ScheduleActionResult { + recentActions := make([]ScheduleActionResult, len(aa)) + for i, a := range aa { + var workflowExecution *ScheduleWorkflowExecution + if a.GetStartWorkflowResult() != nil { + workflowExecution = &ScheduleWorkflowExecution{ + WorkflowID: a.GetStartWorkflowResult().GetWorkflowId(), + FirstExecutionRunID: a.GetStartWorkflowResult().GetRunId(), + } + } + recentActions[i] = ScheduleActionResult{ + ScheduleTime: a.GetScheduleTime().AsTime(), + ActualTime: a.GetActualTime().AsTime(), + StartWorkflowResult: workflowExecution, + } + } + return recentActions +} + +func encodeScheduleWorklowArgs(dc converter.DataConverter, args []interface{}) (*commonpb.Payloads, error) { + payloads := make([]*commonpb.Payload, len(args)) + for i, arg := range args { + // arg is already encoded + if enc, ok := arg.(*commonpb.Payload); ok { + payloads[i] = enc + } else { + payload, err := dc.ToPayload(arg) + if err != nil { + return nil, err + } + payloads[i] = payload + } + } + return &commonpb.Payloads{ + Payloads: payloads, + }, nil +} + +func encodeScheduleWorkflowMemo(dc converter.DataConverter, input map[string]interface{}) (*commonpb.Memo, error) { + if input == nil { + return nil, nil + } + + memo := make(map[string]*commonpb.Payload) + for k, v := range input { + if enc, ok := v.(*commonpb.Payload); ok { + memo[k] = enc + } else { + memoBytes, err := converter.GetDefaultDataConverter().ToPayload(v) + if err != nil { + return nil, fmt.Errorf("encode workflow memo error: %v", err.Error()) + } + memo[k] = memoBytes + } + } + return &commonpb.Memo{Fields: memo}, nil +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_search_attributes.go b/vendor/go.temporal.io/sdk/internal/internal_search_attributes.go new file mode 100644 index 00000000000..6f5e7e25678 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_search_attributes.go @@ -0,0 +1,537 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "fmt" + "reflect" + "time" + + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/log" +) + +type ( + // SearchAttributes represents a collection of typed search attributes + SearchAttributes struct { + untypedValue map[SearchAttributeKey]interface{} + } + + // SearchAttributeUpdate represents a change to SearchAttributes + SearchAttributeUpdate func(*SearchAttributes) + + // SearchAttributeKey represents a typed search attribute key. + SearchAttributeKey interface { + // GetName of the search attribute. + GetName() string + // GetValueType of the search attribute. + GetValueType() enumspb.IndexedValueType + // GetReflectType of the search attribute. + GetReflectType() reflect.Type + } + + baseSearchAttributeKey struct { + name string + valueType enumspb.IndexedValueType + reflectType reflect.Type + } + + // SearchAttributeKeyString represents a search attribute key for a text attribute type. + SearchAttributeKeyString struct { + baseSearchAttributeKey + } + + // SearchAttributeKeyKeyword represents a search attribute key for a keyword attribute type. + SearchAttributeKeyKeyword struct { + baseSearchAttributeKey + } + + // SearchAttributeKeyBool represents a search attribute key for a boolean attribute type. + SearchAttributeKeyBool struct { + baseSearchAttributeKey + } + + // SearchAttributeKeyInt64 represents a search attribute key for a integer attribute type. + SearchAttributeKeyInt64 struct { + baseSearchAttributeKey + } + + // SearchAttributeKeyFloat64 represents a search attribute key for a float attribute type. + SearchAttributeKeyFloat64 struct { + baseSearchAttributeKey + } + + // SearchAttributeKeyTime represents a search attribute key for a date time attribute type. + SearchAttributeKeyTime struct { + baseSearchAttributeKey + } + + // SearchAttributeKeyKeywordList represents a search attribute key for a list of keyword attribute type. + SearchAttributeKeyKeywordList struct { + baseSearchAttributeKey + } +) + +// GetName of the search attribute. +func (bk baseSearchAttributeKey) GetName() string { + return bk.name +} + +// GetValueType of the search attribute. +func (bk baseSearchAttributeKey) GetValueType() enumspb.IndexedValueType { + return bk.valueType +} + +// GetReflectType of the search attribute. +func (bk baseSearchAttributeKey) GetReflectType() reflect.Type { + return bk.reflectType +} + +func NewSearchAttributeKeyString(name string) SearchAttributeKeyString { + return SearchAttributeKeyString{ + baseSearchAttributeKey: baseSearchAttributeKey{ + name: name, + valueType: enumspb.INDEXED_VALUE_TYPE_TEXT, + reflectType: reflect.TypeOf(""), + }, + } +} + +// ValueSet creates an update to set the value of the attribute. +func (k SearchAttributeKeyString) ValueSet(value string) SearchAttributeUpdate { + return func(sa *SearchAttributes) { + sa.untypedValue[k] = value + } +} + +// ValueUnset creates an update to remove the attribute. +func (k SearchAttributeKeyString) ValueUnset() SearchAttributeUpdate { + return func(sa *SearchAttributes) { + sa.untypedValue[k] = nil + } +} + +func NewSearchAttributeKeyKeyword(name string) SearchAttributeKeyKeyword { + return SearchAttributeKeyKeyword{ + baseSearchAttributeKey: baseSearchAttributeKey{ + name: name, + valueType: enumspb.INDEXED_VALUE_TYPE_KEYWORD, + reflectType: reflect.TypeOf(""), + }, + } +} + +// ValueSet creates an update to set the value of the attribute. +func (k SearchAttributeKeyKeyword) ValueSet(value string) SearchAttributeUpdate { + return func(sa *SearchAttributes) { + sa.untypedValue[k] = value + } +} + +// ValueUnset creates an update to remove the attribute. +func (k SearchAttributeKeyKeyword) ValueUnset() SearchAttributeUpdate { + return func(sa *SearchAttributes) { + sa.untypedValue[k] = nil + } +} + +func NewSearchAttributeKeyBool(name string) SearchAttributeKeyBool { + return SearchAttributeKeyBool{ + baseSearchAttributeKey: baseSearchAttributeKey{ + name: name, + valueType: enumspb.INDEXED_VALUE_TYPE_BOOL, + reflectType: reflect.TypeOf(false), + }, + } +} + +// ValueSet creates an update to set the value of the attribute. +func (k SearchAttributeKeyBool) ValueSet(value bool) SearchAttributeUpdate { + return func(sa *SearchAttributes) { + sa.untypedValue[k] = value + } +} + +// ValueUnset creates an update to remove the attribute. +func (k SearchAttributeKeyBool) ValueUnset() SearchAttributeUpdate { + return func(sa *SearchAttributes) { + sa.untypedValue[k] = nil + } +} + +func NewSearchAttributeKeyInt64(name string) SearchAttributeKeyInt64 { + return SearchAttributeKeyInt64{ + baseSearchAttributeKey: baseSearchAttributeKey{ + name: name, + valueType: enumspb.INDEXED_VALUE_TYPE_INT, + reflectType: reflect.TypeOf(int64(0)), + }, + } +} + +// ValueSet creates an update to set the value of the attribute. +func (k SearchAttributeKeyInt64) ValueSet(value int64) SearchAttributeUpdate { + return func(sa *SearchAttributes) { + sa.untypedValue[k] = value + } +} + +// ValueUnset creates an update to remove the attribute. +func (k SearchAttributeKeyInt64) ValueUnset() SearchAttributeUpdate { + return func(sa *SearchAttributes) { + sa.untypedValue[k] = nil + } +} + +func NewSearchAttributeKeyFloat64(name string) SearchAttributeKeyFloat64 { + return SearchAttributeKeyFloat64{ + baseSearchAttributeKey: baseSearchAttributeKey{ + name: name, + valueType: enumspb.INDEXED_VALUE_TYPE_DOUBLE, + reflectType: reflect.TypeOf(float64(0)), + }, + } +} + +// ValueSet creates an update to set the value of the attribute. +func (k SearchAttributeKeyFloat64) ValueSet(value float64) SearchAttributeUpdate { + return func(sa *SearchAttributes) { + sa.untypedValue[k] = value + } +} + +// ValueUnset creates an update to remove the attribute. +func (k SearchAttributeKeyFloat64) ValueUnset() SearchAttributeUpdate { + return func(sa *SearchAttributes) { + sa.untypedValue[k] = nil + } +} + +func NewSearchAttributeKeyTime(name string) SearchAttributeKeyTime { + return SearchAttributeKeyTime{ + baseSearchAttributeKey: baseSearchAttributeKey{ + name: name, + valueType: enumspb.INDEXED_VALUE_TYPE_DATETIME, + reflectType: reflect.TypeOf(time.Time{}), + }, + } +} + +// ValueSet creates an update to set the value of the attribute. +func (k SearchAttributeKeyTime) ValueSet(value time.Time) SearchAttributeUpdate { + return func(sa *SearchAttributes) { + sa.untypedValue[k] = value + } +} + +// ValueUnset creates an update to remove the attribute. +func (k SearchAttributeKeyTime) ValueUnset() SearchAttributeUpdate { + return func(sa *SearchAttributes) { + sa.untypedValue[k] = nil + } +} + +func NewSearchAttributeKeyKeywordList(name string) SearchAttributeKeyKeywordList { + return SearchAttributeKeyKeywordList{ + baseSearchAttributeKey: baseSearchAttributeKey{ + name: name, + valueType: enumspb.INDEXED_VALUE_TYPE_KEYWORD_LIST, + reflectType: reflect.TypeOf([]string{}), + }, + } +} + +// ValueSet creates an update to set the value of the attribute. +func (k SearchAttributeKeyKeywordList) ValueSet(values []string) SearchAttributeUpdate { + listCopy := append([]string(nil), values...) + return func(sa *SearchAttributes) { + sa.untypedValue[k] = listCopy + } +} + +// ValueUnset creates an update to remove the attribute. +func (k SearchAttributeKeyKeywordList) ValueUnset() SearchAttributeUpdate { + return func(sa *SearchAttributes) { + sa.untypedValue[k] = nil + } +} + +func NewSearchAttributes(attributes ...SearchAttributeUpdate) SearchAttributes { + sa := SearchAttributes{ + untypedValue: make(map[SearchAttributeKey]interface{}), + } + for _, attr := range attributes { + attr(&sa) + } + return sa +} + +// GetString gets a value for the given key and whether it was present. +func (sa SearchAttributes) GetString(key SearchAttributeKeyString) (string, bool) { + value, ok := sa.untypedValue[key] + if !ok || value == nil { + return "", false + } + return value.(string), true +} + +// GetKeyword gets a value for the given key and whether it was present. +func (sa SearchAttributes) GetKeyword(key SearchAttributeKeyKeyword) (string, bool) { + value, ok := sa.untypedValue[key] + if !ok || value == nil { + return "", false + } + return value.(string), true +} + +// GetBool gets a value for the given key and whether it was present. +func (sa SearchAttributes) GetBool(key SearchAttributeKeyBool) (bool, bool) { + value, ok := sa.untypedValue[key] + if !ok || value == nil { + return false, false + } + return value.(bool), true +} + +// GetInt64 gets a value for the given key and whether it was present. +func (sa SearchAttributes) GetInt64(key SearchAttributeKeyInt64) (int64, bool) { + value, ok := sa.untypedValue[key] + if !ok || value == nil { + return 0, false + } + return value.(int64), true +} + +// GetFloat64 gets a value for the given key and whether it was present. +func (sa SearchAttributes) GetFloat64(key SearchAttributeKeyFloat64) (float64, bool) { + value, ok := sa.untypedValue[key] + if !ok || value == nil { + return 0.0, false + } + return value.(float64), true +} + +// GetTime gets a value for the given key and whether it was present. +func (sa SearchAttributes) GetTime(key SearchAttributeKeyTime) (time.Time, bool) { + value, ok := sa.untypedValue[key] + if !ok || value == nil { + return time.Time{}, false + } + return value.(time.Time), true +} + +// GetKeywordList gets a value for the given key and whether it was present. +func (sa SearchAttributes) GetKeywordList(key SearchAttributeKeyKeywordList) ([]string, bool) { + value, ok := sa.untypedValue[key] + if !ok || value == nil { + return nil, false + } + result := value.([]string) + // Return a copy to prevent caller from mutating the underlying value + return append([]string(nil), result...), true +} + +// ContainsKey gets whether a key is present. +func (sa SearchAttributes) ContainsKey(key SearchAttributeKey) bool { + val, ok := sa.untypedValue[key] + return ok && val != nil +} + +// Size gets the size of the attribute collection. +func (sa SearchAttributes) Size() int { + return len(sa.GetUntypedValues()) +} + +// GetUntypedValues gets a copy of the collection with raw types. +func (sa SearchAttributes) GetUntypedValues() map[SearchAttributeKey]interface{} { + untypedValueCopy := make(map[SearchAttributeKey]interface{}, len(sa.untypedValue)) + for key, value := range sa.untypedValue { + // Filter out nil values + if value == nil { + continue + } + switch v := value.(type) { + case []string: + untypedValueCopy[key] = append([]string(nil), v...) + default: + untypedValueCopy[key] = v + } + } + return untypedValueCopy +} + +// Copy creates an update that copies existing values. +func (sa SearchAttributes) Copy() SearchAttributeUpdate { + return func(s *SearchAttributes) { + // GetUntypedValues returns a copy of the map without nil values + // so the copy won't delete any existing values + untypedValues := sa.GetUntypedValues() + for key, value := range untypedValues { + s.untypedValue[key] = value + } + } +} + +func serializeUntypedSearchAttributes(input map[string]interface{}) (*commonpb.SearchAttributes, error) { + if input == nil { + return nil, nil + } + + attr := make(map[string]*commonpb.Payload) + for k, v := range input { + // If search attribute value is already of Payload type, then use it directly. + // This allows to copy search attributes from workflow info to child workflow options. + if vp, ok := v.(*commonpb.Payload); ok { + attr[k] = vp + continue + } + var err error + attr[k], err = converter.GetDefaultDataConverter().ToPayload(v) + if err != nil { + return nil, fmt.Errorf("encode search attribute [%s] error: %v", k, err) + } + } + return &commonpb.SearchAttributes{IndexedFields: attr}, nil +} + +func serializeTypedSearchAttributes(searchAttributes map[SearchAttributeKey]interface{}) (*commonpb.SearchAttributes, error) { + if searchAttributes == nil { + return nil, nil + } + + serializedAttr := make(map[string]*commonpb.Payload) + for k, v := range searchAttributes { + payload, err := converter.GetDefaultDataConverter().ToPayload(v) + if err != nil { + return nil, fmt.Errorf("encode search attribute [%s] error: %v", k, err) + } + // Server does not remove search attributes if they set a type + if payload.GetData() != nil { + payload.Metadata["type"] = []byte(k.GetValueType().String()) + } + serializedAttr[k.GetName()] = payload + } + return &commonpb.SearchAttributes{IndexedFields: serializedAttr}, nil +} + +func serializeSearchAttributes( + untypedAttributes map[string]interface{}, + typedAttributes SearchAttributes, +) (*commonpb.SearchAttributes, error) { + var searchAttr *commonpb.SearchAttributes + var err error + if untypedAttributes != nil && typedAttributes.Size() != 0 { + return nil, fmt.Errorf("cannot specify both SearchAttributes and TypedSearchAttributes") + } else if untypedAttributes != nil { + searchAttr, err = serializeUntypedSearchAttributes(untypedAttributes) + if err != nil { + return nil, err + } + } else if typedAttributes.Size() != 0 { + searchAttr, err = serializeTypedSearchAttributes(typedAttributes.GetUntypedValues()) + if err != nil { + return nil, err + } + } + return searchAttr, nil +} + +func convertToTypedSearchAttributes(logger log.Logger, attributes map[string]*commonpb.Payload) SearchAttributes { + updates := make([]SearchAttributeUpdate, 0, len(attributes)) + for key, payload := range attributes { + if payload.Data == nil { + continue + } + valueType := enumspb.IndexedValueType( + enumspb.IndexedValueType_shorthandValue[string(payload.GetMetadata()["type"])]) + // For TemporalChangeVersion, we imply the value type + if valueType == 0 && key == TemporalChangeVersion { + valueType = enumspb.INDEXED_VALUE_TYPE_KEYWORD_LIST + } + switch valueType { + case enumspb.INDEXED_VALUE_TYPE_BOOL: + attr := NewSearchAttributeKeyBool(key) + var value bool + err := converter.GetDefaultDataConverter().FromPayload(payload, &value) + if err != nil { + panic(err) + } + updates = append(updates, attr.ValueSet(value)) + case enumspb.INDEXED_VALUE_TYPE_KEYWORD: + attr := NewSearchAttributeKeyKeyword(key) + var value string + err := converter.GetDefaultDataConverter().FromPayload(payload, &value) + if err != nil { + panic(err) + } + updates = append(updates, attr.ValueSet(value)) + case enumspb.INDEXED_VALUE_TYPE_TEXT: + attr := NewSearchAttributeKeyString(key) + var value string + err := converter.GetDefaultDataConverter().FromPayload(payload, &value) + if err != nil { + panic(err) + } + updates = append(updates, attr.ValueSet(value)) + case enumspb.INDEXED_VALUE_TYPE_INT: + attr := NewSearchAttributeKeyInt64(key) + var value int64 + err := converter.GetDefaultDataConverter().FromPayload(payload, &value) + if err != nil { + panic(err) + } + updates = append(updates, attr.ValueSet(value)) + case enumspb.INDEXED_VALUE_TYPE_DOUBLE: + attr := NewSearchAttributeKeyFloat64(key) + var value float64 + err := converter.GetDefaultDataConverter().FromPayload(payload, &value) + if err != nil { + panic(err) + } + updates = append(updates, attr.ValueSet(value)) + case enumspb.INDEXED_VALUE_TYPE_DATETIME: + attr := NewSearchAttributeKeyTime(key) + var value time.Time + err := converter.GetDefaultDataConverter().FromPayload(payload, &value) + if err != nil { + panic(err) + } + updates = append(updates, attr.ValueSet(value)) + case enumspb.INDEXED_VALUE_TYPE_KEYWORD_LIST: + attr := NewSearchAttributeKeyKeywordList(key) + var value []string + err := converter.GetDefaultDataConverter().FromPayload(payload, &value) + if err != nil { + panic(err) + } + updates = append(updates, attr.ValueSet(value)) + default: + logger.Warn("Unrecognized indexed value type on search attribute key", "key", key, "type", valueType) + } + } + return NewSearchAttributes(updates...) +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_task_handlers.go b/vendor/go.temporal.io/sdk/internal/internal_task_handlers.go new file mode 100644 index 00000000000..83125877685 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_task_handlers.go @@ -0,0 +1,2341 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +// All code in this file is private to the package. + +import ( + "context" + "errors" + "fmt" + "math" + "reflect" + "strings" + "sync" + "time" + + commandpb "go.temporal.io/api/command/v1" + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + historypb "go.temporal.io/api/history/v1" + protocolpb "go.temporal.io/api/protocol/v1" + querypb "go.temporal.io/api/query/v1" + "go.temporal.io/api/sdk/v1" + "go.temporal.io/api/serviceerror" + taskqueuepb "go.temporal.io/api/taskqueue/v1" + "go.temporal.io/api/workflowservice/v1" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/types/known/durationpb" + + "go.temporal.io/sdk/internal/common/retry" + "go.temporal.io/sdk/internal/protocol" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/internal/common/util" + "go.temporal.io/sdk/log" +) + +const ( + defaultStickyCacheSize = 10000 + + noRetryBackoff = time.Duration(-1) + + defaultDefaultHeartbeatThrottleInterval = 30 * time.Second + defaultMaxHeartbeatThrottleInterval = 60 * time.Second +) + +type ( + // workflowExecutionEventHandler process a single event. + workflowExecutionEventHandler interface { + // Process a single event and return the assosciated commands. + // Return List of commands made, any error. + ProcessEvent(event *historypb.HistoryEvent, isReplay bool, isLast bool) error + // ProcessInteraction processes interaction inputs + ProcessMessage(msg *protocolpb.Message, isReplay bool, isLast bool) error + // ProcessQuery process a query request. + ProcessQuery(queryType string, queryArgs *commonpb.Payloads, header *commonpb.Header) (*commonpb.Payloads, error) + StackTrace() string + // Close for cleaning up resources on this event handler + Close() + } + + // workflowTask wraps a workflow task. + workflowTask struct { + task *workflowservice.PollWorkflowTaskQueueResponse + historyIterator HistoryIterator + doneCh chan struct{} + laResultCh chan *localActivityResult + + // This channel must be initialized with a one-size buffer and is used to indicate when + // it is time for a local activity to be retried + laRetryCh chan *localActivityTask + } + + // eagerWorkflowTask represents a workflow task sent from an eager workflow executor + eagerWorkflowTask struct { + task *workflowservice.PollWorkflowTaskQueueResponse + } + + // activityTask wraps a activity task. + activityTask struct { + task *workflowservice.PollActivityTaskQueueResponse + permit *SlotPermit + } + + // workflowExecutionContextImpl is the cached workflow state for sticky execution + workflowExecutionContextImpl struct { + mutex sync.Mutex + workflowInfo *WorkflowInfo + wth *workflowTaskHandlerImpl + + eventHandler *workflowExecutionEventHandler + + isWorkflowCompleted bool + result *commonpb.Payloads + err error + // previousStartedEventID is the event ID of the workflow task started event of the previous workflow task. + previousStartedEventID int64 + // lastHandledEventID is the event ID of the last event that the workflow state machine processed. + lastHandledEventID int64 + + newCommands []*commandpb.Command + newMessages []*protocolpb.Message + currentWorkflowTask *workflowservice.PollWorkflowTaskQueueResponse + laTunnel *localActivityTunnel + cached bool + } + + // workflowTaskHandlerImpl is the implementation of WorkflowTaskHandler + workflowTaskHandlerImpl struct { + namespace string + metricsHandler metrics.Handler + ppMgr pressurePointMgr + logger log.Logger + identity string + workerBuildID string + useBuildIDForVersioning bool + enableLoggingInReplay bool + registry *registry + laTunnel *localActivityTunnel + workflowPanicPolicy WorkflowPanicPolicy + dataConverter converter.DataConverter + failureConverter converter.FailureConverter + contextPropagators []ContextPropagator + cache *WorkerCache + deadlockDetectionTimeout time.Duration + capabilities *workflowservice.GetSystemInfoResponse_Capabilities + } + + activityProvider func(name string) activity + + // activityTaskHandlerImpl is the implementation of ActivityTaskHandler + activityTaskHandlerImpl struct { + taskQueueName string + identity string + service workflowservice.WorkflowServiceClient + metricsHandler metrics.Handler + logger log.Logger + userContext context.Context + registry *registry + activityProvider activityProvider + dataConverter converter.DataConverter + failureConverter converter.FailureConverter + workerStopCh <-chan struct{} + contextPropagators []ContextPropagator + namespace string + defaultHeartbeatThrottleInterval time.Duration + maxHeartbeatThrottleInterval time.Duration + versionStamp *commonpb.WorkerVersionStamp + } + + // history wrapper method to help information about events. + history struct { + workflowTask *workflowTask + eventsHandler *workflowExecutionEventHandlerImpl + loadedEvents []*historypb.HistoryEvent + currentIndex int + nextEventID int64 // next expected eventID for sanity + lastEventID int64 // last expected eventID, zero indicates read until end of stream + lastHandledEventID int64 // last event ID that was processed + next []*historypb.HistoryEvent + nextMessages []*protocolpb.Message + nextFlags []sdkFlag + binaryChecksum string + sdkVersion string + sdkName string + } + + workflowTaskHeartbeatError struct { + Message string + } + + historyMismatchError struct { + message string + } + + unknownSdkFlagError struct { + message string + } + + preparedTask struct { + events []*historypb.HistoryEvent + markers []*historypb.HistoryEvent + flags []sdkFlag + acceptedMsgs []*protocolpb.Message + admittedMsgs []*protocolpb.Message + binaryChecksum string + sdkVersion string + sdkName string + // Is null if there was no task completed event to read the build ID from (but may be + // empty string if there was, and it was empty) + buildID *string + } + + finishedTask struct { + isFailed bool + binaryChecksum string + flags []sdkFlag + sdkVersion string + sdkName string + } +) + +func newHistory(lastHandledEventID int64, task *workflowTask, eventsHandler *workflowExecutionEventHandlerImpl) *history { + result := &history{ + workflowTask: task, + eventsHandler: eventsHandler, + loadedEvents: task.task.History.Events, + currentIndex: 0, + lastEventID: task.task.GetStartedEventId(), + lastHandledEventID: lastHandledEventID, + } + if len(result.loadedEvents) > 0 { + result.nextEventID = result.loadedEvents[0].GetEventId() + } + return result +} + +func (e workflowTaskHeartbeatError) Error() string { + return e.Message +} + +func historyMismatchErrorf(f string, v ...interface{}) historyMismatchError { + return historyMismatchError{message: fmt.Sprintf(f, v...)} +} + +func (h historyMismatchError) Error() string { + return h.message +} + +func (s unknownSdkFlagError) Error() string { + return s.message +} + +// Get workflow start event. +func (eh *history) GetWorkflowStartedEvent() (*historypb.HistoryEvent, error) { + events := eh.workflowTask.task.History.Events + if len(events) == 0 || events[0].GetEventType() != enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED { + return nil, errors.New("unable to find WorkflowExecutionStartedEventAttributes in the history") + } + return events[0], nil +} + +func (eh *history) IsReplayEvent(event *historypb.HistoryEvent) bool { + return event.GetEventId() <= eh.workflowTask.task.GetPreviousStartedEventId() || isCommandEvent(event.GetEventType()) +} + +// isNextWorkflowTaskFailed checks if the workflow task failed or completed. If it did complete returns some information +// on the completed workflow task. +func (eh *history) isNextWorkflowTaskFailed() (task finishedTask, err error) { + nextIndex := eh.currentIndex + 1 + // Server can return an empty page so if we need the next event we must keep checking until we either get it + // or know we have no more pages to check + for nextIndex >= len(eh.loadedEvents) && eh.hasMoreEvents() { // current page ends and there is more pages + if err := eh.loadMoreEvents(); err != nil { + return finishedTask{}, err + } + } + + // If not replaying we should not expect to find any more events + if nextIndex < len(eh.loadedEvents) { + nextEvent := eh.loadedEvents[nextIndex] + nextEventType := nextEvent.GetEventType() + isFailed := nextEventType == enumspb.EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT || nextEventType == enumspb.EVENT_TYPE_WORKFLOW_TASK_FAILED + var binaryChecksum string + var flags []sdkFlag + if nextEventType == enumspb.EVENT_TYPE_WORKFLOW_TASK_COMPLETED { + completedAttrs := nextEvent.GetWorkflowTaskCompletedEventAttributes() + binaryChecksum = completedAttrs.BinaryChecksum + for _, flag := range completedAttrs.GetSdkMetadata().GetLangUsedFlags() { + f := sdkFlagFromUint(flag) + if !f.isValid() { + // If a flag is not recognized (value is too high or not defined), it must fail the workflow task + return finishedTask{}, unknownSdkFlagError{ + message: fmt.Sprintf("unknown SDK flag: %d", flag), + } + } + flags = append(flags, f) + } + } + return finishedTask{ + isFailed: isFailed, + binaryChecksum: binaryChecksum, + flags: flags, + sdkName: nextEvent.GetWorkflowTaskCompletedEventAttributes().GetSdkMetadata().GetSdkName(), + sdkVersion: nextEvent.GetWorkflowTaskCompletedEventAttributes().GetSdkMetadata().GetSdkVersion(), + }, nil + } + return finishedTask{}, nil +} + +func (eh *history) loadMoreEvents() error { + historyPage, err := eh.getMoreEvents() + if err != nil { + return err + } + eh.loadedEvents = append(eh.loadedEvents, historyPage.Events...) + if eh.nextEventID == 0 && len(eh.loadedEvents) > 0 { + eh.nextEventID = eh.loadedEvents[0].GetEventId() + } + return nil +} + +func isCommandEvent(eventType enumspb.EventType) bool { + switch eventType { + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED, + enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_FAILED, + enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CANCELED, + enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW, + enumspb.EVENT_TYPE_ACTIVITY_TASK_SCHEDULED, + enumspb.EVENT_TYPE_ACTIVITY_TASK_CANCEL_REQUESTED, + enumspb.EVENT_TYPE_TIMER_STARTED, + enumspb.EVENT_TYPE_TIMER_CANCELED, + enumspb.EVENT_TYPE_MARKER_RECORDED, + enumspb.EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_INITIATED, + enumspb.EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED, + enumspb.EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED, + enumspb.EVENT_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES, + enumspb.EVENT_TYPE_WORKFLOW_PROPERTIES_MODIFIED, + enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_ACCEPTED, + enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_COMPLETED, + enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_REJECTED, + enumspb.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED, + enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUESTED: + return true + default: + return false + } +} + +// nextTask returns the next task to be processed. +func (eh *history) nextTask() (*preparedTask, error) { + if eh.next == nil { + firstTask, err := eh.prepareTask() + if err != nil { + return nil, err + } + eh.next = firstTask.events + eh.nextMessages = firstTask.admittedMsgs + eh.nextFlags = firstTask.flags + eh.sdkName = firstTask.sdkName + eh.sdkVersion = firstTask.sdkVersion + } + + result := eh.next + requestMessages := eh.nextMessages + checksum := eh.binaryChecksum + sdkFlags := eh.nextFlags + sdkName := eh.sdkName + sdkVersion := eh.sdkVersion + + var markers []*historypb.HistoryEvent + var acceptedMsgs []*protocolpb.Message + var buildID *string + if len(result) > 0 { + nextTaskEvents, err := eh.prepareTask() + if err != nil { + return nil, err + } + eh.next = nextTaskEvents.events + eh.nextMessages = nextTaskEvents.admittedMsgs + eh.nextFlags = nextTaskEvents.flags + eh.sdkName = nextTaskEvents.sdkName + eh.sdkVersion = nextTaskEvents.sdkVersion + markers = nextTaskEvents.markers + acceptedMsgs = nextTaskEvents.acceptedMsgs + buildID = nextTaskEvents.buildID + } + return &preparedTask{ + events: result, + markers: markers, + flags: sdkFlags, + acceptedMsgs: acceptedMsgs, + admittedMsgs: requestMessages, + binaryChecksum: checksum, + sdkName: sdkName, + sdkVersion: sdkVersion, + buildID: buildID, + }, nil +} + +func (eh *history) hasMoreEvents() bool { + historyIterator := eh.workflowTask.historyIterator + return historyIterator != nil && historyIterator.HasNextPage() +} + +func (eh *history) getMoreEvents() (*historypb.History, error) { + return eh.workflowTask.historyIterator.GetNextPage() +} + +func (eh *history) verifyAllEventsProcessed() error { + if eh.lastEventID > 0 && eh.nextEventID <= eh.lastEventID { + return fmt.Errorf( + "history_events: premature end of stream, expectedLastEventID=%v but no more events after eventID=%v", + eh.lastEventID, + eh.nextEventID-1) + } + if eh.lastEventID > 0 && eh.nextEventID != (eh.lastEventID+1) { + eh.eventsHandler.logger.Warn( + "history_events: processed events past the expected lastEventID", + "expectedLastEventID", eh.lastEventID, + "processedLastEventID", eh.nextEventID-1) + } + return nil +} + +func (eh *history) prepareTask() (*preparedTask, error) { + if eh.currentIndex == len(eh.loadedEvents) && !eh.hasMoreEvents() { + if err := eh.verifyAllEventsProcessed(); err != nil { + return nil, err + } + return &preparedTask{}, nil + } + + // Process events + var taskEvents preparedTask +OrderEvents: + for { + // load more history events if needed + for eh.currentIndex == len(eh.loadedEvents) { + if !eh.hasMoreEvents() { + if err := eh.verifyAllEventsProcessed(); err != nil { + return nil, err + } + break OrderEvents + } + if err := eh.loadMoreEvents(); err != nil { + return nil, err + } + } + + event := eh.loadedEvents[eh.currentIndex] + eventID := event.GetEventId() + if eventID != eh.nextEventID { + err := fmt.Errorf( + "missing history events, expectedNextEventID=%v but receivedNextEventID=%v", + eh.nextEventID, eventID) + return nil, err + } + + eh.nextEventID++ + if eventID <= eh.lastHandledEventID { + eh.currentIndex++ + continue + } + eh.lastHandledEventID = eventID + + switch event.GetEventType() { + case enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED: + finishedTask, err1 := eh.isNextWorkflowTaskFailed() + if err1 != nil { + err := err1 + return nil, err + } + if !finishedTask.isFailed { + eh.binaryChecksum = finishedTask.binaryChecksum + eh.currentIndex++ + taskEvents.events = append(taskEvents.events, event) + taskEvents.flags = append(taskEvents.flags, finishedTask.flags...) + if finishedTask.sdkName != "" { + taskEvents.sdkName = finishedTask.sdkName + } + if finishedTask.sdkVersion != "" { + taskEvents.sdkVersion = finishedTask.sdkVersion + } + break OrderEvents + } + case enumspb.EVENT_TYPE_WORKFLOW_TASK_SCHEDULED, + enumspb.EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT, + enumspb.EVENT_TYPE_WORKFLOW_TASK_FAILED: + // Skip + default: + if event.GetEventType() == enumspb.EVENT_TYPE_WORKFLOW_TASK_COMPLETED { + bidStr := event.GetWorkflowTaskCompletedEventAttributes(). + GetWorkerVersion().GetBuildId() + taskEvents.buildID = &bidStr + } else if isPreloadMarkerEvent(event) { + taskEvents.markers = append(taskEvents.markers, event) + } else if attrs := event.GetWorkflowExecutionUpdateAcceptedEventAttributes(); attrs != nil { + taskEvents.acceptedMsgs = append(taskEvents.acceptedMsgs, inferMessageFromAcceptedEvent(attrs)) + } else if attrs := event.GetWorkflowExecutionUpdateAdmittedEventAttributes(); attrs != nil { + updateID := attrs.GetRequest().GetMeta().GetUpdateId() + taskEvents.admittedMsgs = append(taskEvents.admittedMsgs, &protocolpb.Message{ + Id: updateID + "/request", + ProtocolInstanceId: updateID, + SequencingId: &protocolpb.Message_EventId{ + EventId: event.GetEventId(), + }, + Body: protocol.MustMarshalAny(attrs.GetRequest()), + }) + } + taskEvents.events = append(taskEvents.events, event) + } + eh.currentIndex++ + } + + // shrink loaded events so it can be GCed + eh.loadedEvents = append( + make( + []*historypb.HistoryEvent, + 0, + len(eh.loadedEvents)-eh.currentIndex), + eh.loadedEvents[eh.currentIndex:]..., + ) + + eh.currentIndex = 0 + + return &taskEvents, nil +} + +func isPreloadMarkerEvent(event *historypb.HistoryEvent) bool { + return event.GetEventType() == enumspb.EVENT_TYPE_MARKER_RECORDED +} + +func inferMessageFromAcceptedEvent(attrs *historypb.WorkflowExecutionUpdateAcceptedEventAttributes) *protocolpb.Message { + return &protocolpb.Message{ + Id: attrs.GetAcceptedRequestMessageId(), + ProtocolInstanceId: attrs.GetProtocolInstanceId(), + SequencingId: &protocolpb.Message_EventId{ + EventId: attrs.GetAcceptedRequestSequencingEventId(), + }, + Body: protocol.MustMarshalAny(attrs.GetAcceptedRequest()), + } +} + +// newWorkflowTaskHandler returns an implementation of workflow task handler. +func newWorkflowTaskHandler(params workerExecutionParameters, ppMgr pressurePointMgr, registry *registry) WorkflowTaskHandler { + ensureRequiredParams(¶ms) + return &workflowTaskHandlerImpl{ + namespace: params.Namespace, + logger: params.Logger, + ppMgr: ppMgr, + metricsHandler: params.MetricsHandler, + identity: params.Identity, + workerBuildID: params.getBuildID(), + useBuildIDForVersioning: params.UseBuildIDForVersioning, + enableLoggingInReplay: params.EnableLoggingInReplay, + registry: registry, + workflowPanicPolicy: params.WorkflowPanicPolicy, + dataConverter: params.DataConverter, + failureConverter: params.FailureConverter, + contextPropagators: params.ContextPropagators, + cache: params.cache, + deadlockDetectionTimeout: params.DeadlockDetectionTimeout, + capabilities: params.capabilities, + } +} + +func newWorkflowExecutionContext( + workflowInfo *WorkflowInfo, + taskHandler *workflowTaskHandlerImpl, +) *workflowExecutionContextImpl { + workflowContext := &workflowExecutionContextImpl{ + workflowInfo: workflowInfo, + wth: taskHandler, + } + workflowContext.createEventHandler() + return workflowContext +} + +// Lock acquires the lock on this context object, use Unlock(error) to release +// the lock. +func (w *workflowExecutionContextImpl) Lock() { + w.mutex.Lock() +} + +// Unlock cleans up after the provided error and it's own internal view of the +// workflow error state by clearing itself and removing itself from cache as +// needed. It is an error to call this function without having called the Lock +// function first and the behavior is undefined. Regardless of the error +// handling involved, the context will be unlocked when this call returns. +func (w *workflowExecutionContextImpl) Unlock(err error) { + defer w.mutex.Unlock() + if err != nil || w.err != nil || w.isWorkflowCompleted || + (w.wth.cache.MaxWorkflowCacheSize() <= 0 && !w.hasPendingLocalActivityWork()) { + // TODO: in case of closed, it assumes the close command always succeed. need server side change to return + // error to indicate the close failure case. This should be rare case. For now, always remove the cache, and + // if the close command failed, the next command will have to rebuild the state. + if w.wth.cache.getWorkflowCache().Exist(w.workflowInfo.WorkflowExecution.RunID) { + w.wth.cache.removeWorkflowContext(w.workflowInfo.WorkflowExecution.RunID) + w.cached = false + } + // Clear the state so other tasks waiting on the context know it should be discarded. + w.clearState() + } else if !w.cached { + // Clear the state if we never cached the workflow so coroutines can be + // exited + w.clearState() + } +} + +func (w *workflowExecutionContextImpl) getEventHandler() *workflowExecutionEventHandlerImpl { + if w.eventHandler == nil { + return nil + } + return (*w.eventHandler).(*workflowExecutionEventHandlerImpl) +} + +func (w *workflowExecutionContextImpl) completeWorkflow(result *commonpb.Payloads, err error) { + w.isWorkflowCompleted = true + w.result = result + w.err = err +} + +func (w *workflowExecutionContextImpl) onEviction() { + // onEviction is run by LRU cache's removeFunc in separate goroutinue + w.mutex.Lock() + + // Emit force eviction metrics. + // This metrics indicates too many concurrent running workflows to fit in sticky cache. + // Eviction on error or on workflow complete is normal and expected. + if w.err == nil && !w.isWorkflowCompleted { + w.wth.metricsHandler.Counter(metrics.StickyCacheTotalForcedEviction).Inc(1) + } + + w.clearState() + w.mutex.Unlock() +} + +func (w *workflowExecutionContextImpl) IsDestroyed() bool { + return w.getEventHandler() == nil +} + +func (w *workflowExecutionContextImpl) clearState() { + w.clearCurrentTask() + w.isWorkflowCompleted = false + w.result = nil + w.err = nil + w.previousStartedEventID = 0 + w.lastHandledEventID = 0 + w.newCommands = nil + w.newMessages = nil + + eventHandler := w.getEventHandler() + if eventHandler != nil { + // Set isReplay to true to prevent user code in defer guarded by !isReplaying() from running + eventHandler.isReplay = true + eventHandler.Close() + w.eventHandler = nil + } +} + +func (w *workflowExecutionContextImpl) createEventHandler() { + w.clearState() + eventHandler := newWorkflowExecutionEventHandler( + w.workflowInfo, + w.completeWorkflow, + w.wth.logger, + w.wth.enableLoggingInReplay, + w.wth.metricsHandler, + w.wth.registry, + w.wth.dataConverter, + w.wth.failureConverter, + w.wth.contextPropagators, + w.wth.deadlockDetectionTimeout, + w.wth.capabilities, + ) + + w.eventHandler = &eventHandler +} + +func resetHistory(task *workflowservice.PollWorkflowTaskQueueResponse, historyIterator HistoryIterator) (*historypb.History, error) { + historyIterator.Reset() + firstPageHistory, err := historyIterator.GetNextPage() + if err != nil { + return nil, err + } + task.History = firstPageHistory + return firstPageHistory, nil +} + +func (wth *workflowTaskHandlerImpl) createWorkflowContext(task *workflowservice.PollWorkflowTaskQueueResponse) (*workflowExecutionContextImpl, error) { + h := task.History + startedEvent := h.Events[0] + attributes := startedEvent.GetWorkflowExecutionStartedEventAttributes() + if attributes == nil { + return nil, errors.New("first history event is not WorkflowExecutionStarted") + } + taskQueue := attributes.TaskQueue + if taskQueue == nil || taskQueue.Name == "" { + return nil, errors.New("nil or empty TaskQueue in WorkflowExecutionStarted event") + } + + runID := task.WorkflowExecution.GetRunId() + workflowID := task.WorkflowExecution.GetWorkflowId() + + // Setup workflow Info + var parentWorkflowExecution *WorkflowExecution + if attributes.ParentWorkflowExecution != nil { + parentWorkflowExecution = &WorkflowExecution{ + ID: attributes.ParentWorkflowExecution.GetWorkflowId(), + RunID: attributes.ParentWorkflowExecution.GetRunId(), + } + } + workflowInfo := &WorkflowInfo{ + WorkflowExecution: WorkflowExecution{ + ID: workflowID, + RunID: runID, + }, + OriginalRunID: attributes.OriginalExecutionRunId, + FirstRunID: attributes.FirstExecutionRunId, + WorkflowType: WorkflowType{Name: task.WorkflowType.GetName()}, + TaskQueueName: taskQueue.GetName(), + WorkflowExecutionTimeout: attributes.GetWorkflowExecutionTimeout().AsDuration(), + WorkflowRunTimeout: attributes.GetWorkflowRunTimeout().AsDuration(), + WorkflowTaskTimeout: attributes.GetWorkflowTaskTimeout().AsDuration(), + Namespace: wth.namespace, + Attempt: attributes.GetAttempt(), + WorkflowStartTime: startedEvent.GetEventTime().AsTime(), + lastCompletionResult: attributes.LastCompletionResult, + lastFailure: attributes.ContinuedFailure, + CronSchedule: attributes.CronSchedule, + ContinuedExecutionRunID: attributes.ContinuedExecutionRunId, + ParentWorkflowNamespace: attributes.ParentWorkflowNamespace, + ParentWorkflowExecution: parentWorkflowExecution, + Memo: attributes.Memo, + SearchAttributes: attributes.SearchAttributes, + RetryPolicy: convertFromPBRetryPolicy(attributes.RetryPolicy), + } + + return newWorkflowExecutionContext(workflowInfo, wth), nil +} + +func (wth *workflowTaskHandlerImpl) GetOrCreateWorkflowContext( + task *workflowservice.PollWorkflowTaskQueueResponse, + historyIterator HistoryIterator, +) (workflowContext *workflowExecutionContextImpl, err error) { + metricsHandler := wth.metricsHandler.WithTags(metrics.WorkflowTags(task.WorkflowType.GetName())) + defer func() { + if err == nil && workflowContext != nil && workflowContext.laTunnel == nil { + workflowContext.laTunnel = wth.laTunnel + } + metricsHandler.Gauge(metrics.StickyCacheSize).Update(float64(wth.cache.getWorkflowCache().Size())) + }() + + runID := task.WorkflowExecution.GetRunId() + + history := task.History + isFullHistory := isFullHistory(history) + + workflowContext = nil + if task.Query == nil || (task.Query != nil && !isFullHistory) { + workflowContext = wth.cache.getWorkflowContext(runID) + } + // Verify the cached state is current and for the correct worker + if workflowContext != nil { + workflowContext.Lock() + if task.Query != nil && !isFullHistory && wth == workflowContext.wth && !workflowContext.IsDestroyed() { + // query task and we have a valid cached state + metricsHandler.Counter(metrics.StickyCacheHit).Inc(1) + } else if len(history.Events) > 0 && history.Events[0].GetEventId() == workflowContext.previousStartedEventID+1 && wth == workflowContext.wth && !workflowContext.IsDestroyed() { + // non query task and we have a valid cached state + metricsHandler.Counter(metrics.StickyCacheHit).Inc(1) + } else { + // possible another task already destroyed this context. + if !workflowContext.IsDestroyed() { + // non query task and cached state is missing events, we need to discard the cached state and build a new one. + if len(history.Events) > 0 && history.Events[0].GetEventId() != workflowContext.previousStartedEventID+1 { + wth.logger.Debug("Cached state staled, new task has unexpected events", + tagWorkflowID, task.WorkflowExecution.GetWorkflowId(), + tagRunID, task.WorkflowExecution.GetRunId(), + tagAttempt, task.Attempt, + tagCachedPreviousStartedEventID, workflowContext.previousStartedEventID, + tagTaskFirstEventID, task.History.Events[0].GetEventId(), + tagTaskStartedEventID, task.GetStartedEventId(), + tagPreviousStartedEventID, task.GetPreviousStartedEventId(), + ) + } else { + wth.logger.Debug("Cached state started on different worker, creating new context") + } + wth.cache.removeWorkflowContext(runID) + workflowContext.clearState() + } + workflowContext.Unlock(err) + workflowContext = nil + } + } + // If the workflow was not cached or the cache was stale. + if workflowContext == nil { + if !isFullHistory { + // we are getting partial history task, but cached state was already evicted. + // we need to reset history so we get events from beginning to replay/rebuild the state + metricsHandler.Counter(metrics.StickyCacheMiss).Inc(1) + if _, err = resetHistory(task, historyIterator); err != nil { + return + } + } + + if workflowContext, err = wth.createWorkflowContext(task); err != nil { + return + } + + if wth.cache.MaxWorkflowCacheSize() > 0 && task.Query == nil { + workflowContext, _ = wth.cache.putWorkflowContext(runID, workflowContext) + workflowContext.Lock() + workflowContext.cached = true + } else { + workflowContext.Lock() + } + } + + err = workflowContext.resetStateIfDestroyed(task, historyIterator) + if err != nil { + workflowContext.Unlock(err) + } + + return +} + +func isFullHistory(history *historypb.History) bool { + if len(history.Events) == 0 || history.Events[0].GetEventType() != enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED { + return false + } + return true +} + +func (w *workflowExecutionContextImpl) resetStateIfDestroyed(task *workflowservice.PollWorkflowTaskQueueResponse, historyIterator HistoryIterator) error { + // It is possible that 2 threads (one for workflow task and one for query task) that both are getting this same + // cached workflowContext. If one task finished with err, it would destroy the cached state. In that case, the + // second task needs to reset the cache state and start from beginning of the history. + if w.IsDestroyed() { + w.createEventHandler() + // reset history events if necessary + if !isFullHistory(task.History) { + if _, err := resetHistory(task, historyIterator); err != nil { + return err + } + } + if w.workflowInfo != nil { + // Reset the search attributes and memos from the WorkflowExecutionStartedEvent. + // The search attributes and memo may have been modified by calls like UpsertMemo + // or UpsertSearchAttributes. They must be reset to avoid non determinism on replay. + h := task.History + startedEvent := h.Events[0] + attributes := startedEvent.GetWorkflowExecutionStartedEventAttributes() + if attributes == nil { + return errors.New("first history event is not WorkflowExecutionStarted") + } + w.workflowInfo.SearchAttributes = attributes.SearchAttributes + w.workflowInfo.Memo = attributes.Memo + } + } + return nil +} + +// ProcessWorkflowTask processes all the events of the workflow task. +func (wth *workflowTaskHandlerImpl) ProcessWorkflowTask( + workflowTask *workflowTask, + workflowContext *workflowExecutionContextImpl, + heartbeatFunc workflowTaskHeartbeatFunc, +) (completeRequest interface{}, errRet error) { + if workflowTask == nil || workflowTask.task == nil { + return nil, errors.New("nil workflow task provided") + } + task := workflowTask.task + if task.History == nil || len(task.History.Events) == 0 { + task.History = &historypb.History{ + Events: []*historypb.HistoryEvent{}, + } + } + if task.Query == nil && len(task.History.Events) == 0 { + return nil, errors.New("nil or empty history") + } + + if task.Query != nil && len(task.Queries) != 0 { + return nil, errors.New("invalid query workflow task") + } + + runID := task.WorkflowExecution.GetRunId() + workflowID := task.WorkflowExecution.GetWorkflowId() + traceLog(func() { + wth.logger.Debug("Processing new workflow task.", + tagWorkflowType, task.WorkflowType.GetName(), + tagWorkflowID, workflowID, + tagRunID, runID, + tagAttempt, task.Attempt, + tagPreviousStartedEventID, task.GetPreviousStartedEventId()) + }) + + var ( + response interface{} + err error + heartbeatTimer *time.Timer + ) + + defer func() { + if heartbeatTimer != nil { + heartbeatTimer.Stop() + } + }() + +processWorkflowLoop: + for { + startTime := time.Now() + response, err = workflowContext.ProcessWorkflowTask(workflowTask) + if err == nil && response == nil { + waitLocalActivityLoop: + for { + deadlineToTrigger := time.Duration(float32(ratioToForceCompleteWorkflowTaskComplete) * float32(workflowContext.workflowInfo.WorkflowTaskTimeout)) + delayDuration := time.Until(startTime.Add(deadlineToTrigger)) + + heartbeatLoop: + for { + if delayDuration <= 0 { + if heartbeatTimer != nil { + heartbeatTimer.Stop() + heartbeatTimer = nil + } + + // force complete, call the workflow task heartbeat function + workflowTask, err = heartbeatFunc( + workflowContext.CompleteWorkflowTask(workflowTask, false), + startTime, + ) + if err != nil { + errRet = &workflowTaskHeartbeatError{Message: fmt.Sprintf("error sending workflow task heartbeat %v", err)} + return + } + if workflowTask == nil { + return + } + + continue processWorkflowLoop + } + + if heartbeatTimer == nil { + heartbeatTimer = time.NewTimer(delayDuration) + } + + select { + case <-heartbeatTimer.C: + delayDuration = 0 + continue heartbeatLoop + + case laRetry := <-workflowTask.laRetryCh: + eventHandler := workflowContext.getEventHandler() + + // if workflow task heartbeat failed, the workflow execution context will be cleared and eventHandler will be nil + if eventHandler == nil { + break processWorkflowLoop + } + + if _, ok := eventHandler.pendingLaTasks[laRetry.activityID]; !ok { + break processWorkflowLoop + } + + laRetry.attempt++ + + if !wth.laTunnel.sendTask(laRetry) { + laRetry.attempt-- + } + + case lar := <-workflowTask.laResultCh: + // local activity result ready + response, err = workflowContext.ProcessLocalActivityResult(workflowTask, lar) + if err == nil && response == nil { + // workflow task is not done yet, still waiting for more local activities + continue waitLocalActivityLoop + } + break processWorkflowLoop + } + } + } + } else { + break processWorkflowLoop + } + } + errRet = err + completeRequest = response + return +} + +func (w *workflowExecutionContextImpl) ProcessWorkflowTask(workflowTask *workflowTask) (interface{}, error) { + task := workflowTask.task + historyIterator := workflowTask.historyIterator + if err := w.ResetIfStale(task, historyIterator); err != nil { + return nil, err + } + w.SetCurrentTask(task) + + eventHandler := w.getEventHandler() + reorderedHistory := newHistory(w.lastHandledEventID, workflowTask, eventHandler) + defer func() { + // After processing the workflow task, update the last handled event ID + // to the last event ID in the history. We do this regardless of whether the workflow task + // was successfully processed or not. This is because a failed workflow task will cause the + // cache to be evicted and the next workflow task will start from the beginning of the history. + w.lastHandledEventID = reorderedHistory.lastHandledEventID + }() + var replayOutbox []outboxEntry + var replayCommands []*commandpb.Command + var respondEvents []*historypb.HistoryEvent + + taskMessages := workflowTask.task.GetMessages() + skipReplayCheck := w.skipReplayCheck() + shouldForceReplayCheck := func() bool { + isInReplayer := IsReplayNamespace(w.wth.namespace) + // If we are in the replayer we should always check the history replay, even if the workflow is completed + // Skip if the workflow panicked to avoid potentially breaking old histories + _, wfPanicked := w.err.(*workflowPanicError) + return !wfPanicked && isInReplayer + } + + metricsHandler := w.wth.metricsHandler.WithTags(metrics.WorkflowTags(task.WorkflowType.GetName())) + start := time.Now() + // This is set to nil once recorded + metricsTimer := metricsHandler.Timer(metrics.WorkflowTaskReplayLatency) + + eventHandler.ResetLAWFTAttemptCounts() + eventHandler.sdkFlags.markSDKFlagsSent() + + w.workflowInfo.currentTaskBuildID = w.wth.workerBuildID +ProcessEvents: + for { + nextTask, err := reorderedHistory.nextTask() + if err != nil { + return nil, err + } + reorderedEvents := nextTask.events + markers := nextTask.markers + historyMessages := nextTask.acceptedMsgs + flags := nextTask.flags + binaryChecksum := nextTask.binaryChecksum + nextTaskBuildId := nextTask.buildID + admittedUpdates := nextTask.admittedMsgs + // Check if we are replaying so we know if we should use the messages in the WFT or the history + isReplay := len(reorderedEvents) > 0 && reorderedHistory.IsReplayEvent(reorderedEvents[len(reorderedEvents)-1]) + var msgs *eventMsgIndex + if isReplay { + admittedUpdatesByID := make(map[string]*protocolpb.Message, len(admittedUpdates)) + for _, admittedUpdate := range admittedUpdates { + admittedUpdatesByID[admittedUpdate.GetProtocolInstanceId()] = admittedUpdate + } + // Check if we need to replace the update message synthesize from an + // accepted event with the update message synthesize from an admitted event + for i, msg := range historyMessages { + if admittedUpdate, ok := admittedUpdatesByID[msg.GetProtocolInstanceId()]; ok { + historyMessages[i] = admittedUpdate + } + // At this point, all update messages should have a body + if historyMessages[i].Body == nil { + return nil, fmt.Errorf("missing body in message for update ID %v", msg.GetProtocolInstanceId()) + } + } + msgs = indexMessagesByEventID(historyMessages) + + eventHandler.sdkVersion = nextTask.sdkVersion + eventHandler.sdkName = nextTask.sdkName + } else { + taskMessages = append(taskMessages, admittedUpdates...) + msgs = indexMessagesByEventID(taskMessages) + taskMessages = []*protocolpb.Message{} + if eventHandler.sdkVersion != SDKVersion { + eventHandler.sdkVersionUpdated = true + eventHandler.sdkVersion = SDKVersion + } + if eventHandler.sdkName != SDKName { + eventHandler.sdkNameUpdated = true + eventHandler.sdkName = SDKName + } + } + + eventHandler.sdkFlags.set(flags...) + if len(reorderedEvents) == 0 { + break ProcessEvents + } + if binaryChecksum == "" { + w.workflowInfo.BinaryChecksum = w.wth.workerBuildID + } else { + w.workflowInfo.BinaryChecksum = binaryChecksum + } + if isReplay && nextTaskBuildId != nil { + w.workflowInfo.currentTaskBuildID = *nextTaskBuildId + } + // Reset the mutable side effect markers recorded + eventHandler.mutableSideEffectsRecorded = nil + // Markers are from the events that are produced from the current workflow task. + for _, m := range markers { + if m.GetMarkerRecordedEventAttributes().GetMarkerName() != localActivityMarkerName { + // local activity marker needs to be applied after workflow task started event + err := eventHandler.ProcessEvent(m, true, false) + if err != nil { + return nil, err + } + if w.isWorkflowCompleted && !shouldForceReplayCheck() { + break ProcessEvents + } + } + } + + for i, event := range reorderedEvents { + isInReplay := reorderedHistory.IsReplayEvent(event) + if !isInReplay && metricsTimer != nil { + metricsTimer.Record(time.Since(start)) + metricsTimer = nil + } + + isLast := !isInReplay && i == len(reorderedEvents)-1 + if !skipReplayCheck && isCommandEvent(event.GetEventType()) { + respondEvents = append(respondEvents, event) + } + + if isPreloadMarkerEvent(event) { + // marker events are processed separately + continue + } + + // Any pressure points. + err := w.wth.executeAnyPressurePoints(event, isInReplay) + if err != nil { + return nil, err + } + + // because we don't run all events through this code path, we have + // to run ProcessMessages both before and after ProcessEvent to + // catch any messages that should have been delivered _before_ this + // event but perhaps were not because there were attached to an + // event (e.g. WFTScheduledEvent) that does not come through this + // loop. + for _, msg := range msgs.takeLTE(event.GetEventId() - 1) { + err := eventHandler.ProcessMessage(msg, isInReplay, isLast) + if err != nil { + return nil, err + } + if w.isWorkflowCompleted && !shouldForceReplayCheck() { + break ProcessEvents + } + } + + err = eventHandler.ProcessEvent(event, isInReplay, isLast) + if err != nil { + return nil, err + } + if w.isWorkflowCompleted && !shouldForceReplayCheck() { + break ProcessEvents + } + + for _, msg := range msgs.takeLTE(event.GetEventId()) { + err := eventHandler.ProcessMessage(msg, isInReplay, isLast) + if err != nil { + return nil, err + } + if w.isWorkflowCompleted && !shouldForceReplayCheck() { + break ProcessEvents + } + } + } + + // now apply local activity markers + for _, m := range markers { + if m.GetMarkerRecordedEventAttributes().GetMarkerName() == localActivityMarkerName { + err := eventHandler.ProcessEvent(m, true, false) + if err != nil { + return nil, err + } + if w.isWorkflowCompleted && !shouldForceReplayCheck() { + break ProcessEvents + } + } + } + if isReplay { + eventCommands := eventHandler.commandsHelper.getCommands(true) + if !skipReplayCheck { + replayCommands = append(replayCommands, eventCommands...) + replayOutbox = append(replayOutbox, eventHandler.outbox...) + } + eventHandler.outbox = nil + } + } + + if metricsTimer != nil { + metricsTimer.Record(time.Since(start)) + metricsTimer = nil + } + + // Non-deterministic error could happen in 2 different places: + // 1) the replay commands does not match to history events. This is usually due to non backwards compatible code + // change to workflow logic. For example, change calling one activity to a different activity. + // 2) the command state machine is trying to make illegal state transition while replay a history event (like + // activity task completed), but the corresponding workflow code that start the event has been removed. In that case + // the replay of that event will panic on the command state machine and the workflow will be marked as completed + // with the panic error. + var workflowError error + if !skipReplayCheck && (!w.isWorkflowCompleted || shouldForceReplayCheck()) { + // check if commands from reply matches to the history events + if err := matchReplayWithHistory(replayCommands, respondEvents, replayOutbox, w.getEventHandler().sdkFlags); err != nil { + workflowError = err + w.err = err + } + } + + return w.applyWorkflowPanicPolicy(workflowTask, workflowError) +} + +func (w *workflowExecutionContextImpl) ProcessLocalActivityResult(workflowTask *workflowTask, lar *localActivityResult) (interface{}, error) { + if lar.err != nil && w.retryLocalActivity(lar) { + return nil, nil // nothing to do here as we are retrying... + } + + return w.applyWorkflowPanicPolicy(workflowTask, w.getEventHandler().ProcessLocalActivityResult(lar)) +} + +func (w *workflowExecutionContextImpl) applyWorkflowPanicPolicy(workflowTask *workflowTask, workflowError error) (interface{}, error) { + task := workflowTask.task + + if workflowError == nil && w.err != nil { + if panicErr, ok := w.err.(*workflowPanicError); ok { + workflowError = panicErr + } + } + + if workflowError != nil { + if panicErr, ok := w.err.(*workflowPanicError); ok { + w.wth.logger.Error("Workflow panic", + tagWorkflowType, task.WorkflowType.GetName(), + tagWorkflowID, task.WorkflowExecution.GetWorkflowId(), + tagRunID, task.WorkflowExecution.GetRunId(), + tagAttempt, task.Attempt, + tagError, workflowError, + tagStackTrace, panicErr.StackTrace()) + } else { + w.wth.logger.Error("Workflow panic", + tagWorkflowType, task.WorkflowType.GetName(), + tagWorkflowID, task.WorkflowExecution.GetWorkflowId(), + tagRunID, task.WorkflowExecution.GetRunId(), + tagAttempt, task.Attempt, + tagError, workflowError) + } + + switch w.wth.workflowPanicPolicy { + case FailWorkflow: + // complete workflow with custom error will fail the workflow + w.getEventHandler().Complete(nil, NewApplicationError( + "Workflow failed on panic due to FailWorkflow workflow panic policy", + "", false, workflowError)) + case BlockWorkflow: + // return error here will be convert to WorkflowTaskFailed for the first time, and ignored for subsequent + // attempts which will cause WorkflowTaskTimeout and server will retry forever until issue got fixed or + // workflow timeout. + return nil, workflowError + default: + panic("unknown mismatched workflow history policy.") + } + } + + return w.CompleteWorkflowTask(workflowTask, true), nil +} + +func (w *workflowExecutionContextImpl) retryLocalActivity(lar *localActivityResult) bool { + if lar.task.retryPolicy == nil || lar.err == nil || IsCanceledError(lar.err) { + return false + } + + retryBackoff := getRetryBackoff(lar, time.Now()) + if retryBackoff > 0 && retryBackoff <= w.workflowInfo.WorkflowTaskTimeout { + // we need a local retry + time.AfterFunc(retryBackoff, func() { + // Send retry signal + select { + case lar.task.workflowTask.laRetryCh <- lar.task: + case <-lar.task.workflowTask.doneCh: + // Task is already done. Abort retrying. + } + }) + return true + } + // Backoff could be large and potentially much larger than WorkflowTaskTimeout. We cannot just sleep locally for + // retry. Because it will delay the local activity from complete which keeps the workflow task open. In order to + // keep workflow task open, we have to keep "heartbeating" current workflow task. + // In that case, it is more efficient to create a server timer with backoff duration and retry when that backoff + // timer fires. So here we will return false to indicate we don't need local retry anymore. However, we have to + // store the current attempt and backoff to the same LocalActivityResultMarker so the replay can do the right thing. + // The backoff timer will be created by workflow.ExecuteLocalActivity(). + lar.backoff = retryBackoff + + return false +} + +func getRetryBackoff(lar *localActivityResult, now time.Time) time.Duration { + return getRetryBackoffWithNowTime(lar.task.retryPolicy, lar.task.attempt, lar.err, now, lar.task.expireTime) +} + +func getRetryBackoffWithNowTime(p *RetryPolicy, attempt int32, err error, now, expireTime time.Time) time.Duration { + if !IsRetryable(err, p.NonRetryableErrorTypes) { + return noRetryBackoff + } + + if p.MaximumAttempts > 0 && attempt >= p.MaximumAttempts { + return noRetryBackoff // max attempt reached + } + + var backoffInterval time.Duration + // Extract backoff interval from error if it is a retryable error. + // Not using errors.As() since we don't want to explore the whole error chain. + if applicationErr, ok := err.(*ApplicationError); ok { + backoffInterval = applicationErr.nextRetryDelay + } + // Calculate next backoff interval if the error did not contain the next backoff interval. + // attempt starts from 1 + if backoffInterval == 0 { + backoffInterval = time.Duration(float64(p.InitialInterval) * math.Pow(p.BackoffCoefficient, float64(attempt-1))) + if backoffInterval <= 0 { + // math.Pow() could overflow + if p.MaximumInterval > 0 { + backoffInterval = p.MaximumInterval + } + } + if p.MaximumInterval > 0 && backoffInterval > p.MaximumInterval { + // cap next interval to MaxInterval + backoffInterval = p.MaximumInterval + } + } + if backoffInterval <= 0 { + return noRetryBackoff + } + + nextScheduleTime := now.Add(backoffInterval) + if !expireTime.IsZero() && nextScheduleTime.After(expireTime) { + return noRetryBackoff + } + + return backoffInterval +} + +func (w *workflowExecutionContextImpl) CompleteWorkflowTask(workflowTask *workflowTask, waitLocalActivities bool) interface{} { + if w.currentWorkflowTask == nil { + return nil + } + eventHandler := w.getEventHandler() + + // w.laTunnel could be nil for worker.ReplayHistory() because there is no worker started, in that case we don't + // care about the pending local activities, and just return because the result is ignored anyway by the caller. + if w.hasPendingLocalActivityWork() && w.laTunnel != nil { + if len(eventHandler.unstartedLaTasks) > 0 { + // start new local activity tasks + unstartedLaTasks := make(map[string]struct{}) + for activityID := range eventHandler.unstartedLaTasks { + task := eventHandler.pendingLaTasks[activityID] + task.wc = w + task.workflowTask = workflowTask + + task.scheduledTime = time.Now() + + if !w.laTunnel.sendTask(task) { + unstartedLaTasks[activityID] = struct{}{} + task.wc = nil + task.workflowTask = nil + } + } + eventHandler.unstartedLaTasks = unstartedLaTasks + } + // cannot complete workflow task as there are pending local activities + if waitLocalActivities { + return nil + } + } + + eventCommands := eventHandler.commandsHelper.getCommands(true) + if len(eventCommands) > 0 { + w.newCommands = append(w.newCommands, eventCommands...) + } + + w.newMessages = append(w.newMessages, eventHandler.takeOutgoingMessages()...) + eventHandler.protocols.ClearCompleted() + + completeRequest := w.wth.completeWorkflow(eventHandler, w.currentWorkflowTask, w, w.newCommands, w.newMessages, !waitLocalActivities) + w.clearCurrentTask() + + return completeRequest +} + +func (w *workflowExecutionContextImpl) hasPendingLocalActivityWork() bool { + eventHandler := w.getEventHandler() + return !w.isWorkflowCompleted && + w.currentWorkflowTask != nil && + w.currentWorkflowTask.Query == nil && // don't run local activity for query task + eventHandler != nil && + len(eventHandler.pendingLaTasks) > 0 +} + +func (w *workflowExecutionContextImpl) clearCurrentTask() { + w.newCommands = nil + w.newMessages = nil + w.currentWorkflowTask = nil +} + +func (w *workflowExecutionContextImpl) skipReplayCheck() bool { + return w.currentWorkflowTask.Query != nil || !isFullHistory(w.currentWorkflowTask.History) +} + +func (w *workflowExecutionContextImpl) SetCurrentTask(task *workflowservice.PollWorkflowTaskQueueResponse) { + w.currentWorkflowTask = task + // do not update the previousStartedEventID for query task + if task.Query == nil { + w.previousStartedEventID = task.GetStartedEventId() + } +} + +func (w *workflowExecutionContextImpl) SetPreviousStartedEventID(eventID int64) { + // We must reset the last event we handled to be after the last WFT we really completed + // + any command events (since the SDK "processed" those when it emitted the commands). This + // is also equal to what we just processed in the speculative task, minus two, since we + // would've just handled the most recent WFT started event, and we need to drop that & the + // schedule event just before it. + w.lastHandledEventID = w.lastHandledEventID - 2 + w.previousStartedEventID = eventID +} + +func (w *workflowExecutionContextImpl) ResetIfStale(task *workflowservice.PollWorkflowTaskQueueResponse, historyIterator HistoryIterator) error { + if len(task.History.Events) > 0 && task.History.Events[0].GetEventId() != w.previousStartedEventID+1 { + w.wth.logger.Debug("Cached state staled, new task has unexpected events", + tagWorkflowID, task.WorkflowExecution.GetWorkflowId(), + tagRunID, task.WorkflowExecution.GetRunId(), + tagAttempt, task.Attempt, + tagCachedPreviousStartedEventID, w.previousStartedEventID, + tagTaskFirstEventID, task.History.Events[0].GetEventId(), + tagTaskStartedEventID, task.GetStartedEventId(), + tagPreviousStartedEventID, task.GetPreviousStartedEventId(), + ) + w.clearState() + return w.resetStateIfDestroyed(task, historyIterator) + } + return nil +} + +func skipDeterministicCheckForCommand(d *commandpb.Command, _ *sdkFlags) bool { + switch d.GetCommandType() { + case enumspb.COMMAND_TYPE_RECORD_MARKER: + markerName := d.GetRecordMarkerCommandAttributes().GetMarkerName() + if markerName == versionMarkerName || markerName == mutableSideEffectMarkerName { + return true + } + } + return false +} + +func skipDeterministicCheckForEvent(e *historypb.HistoryEvent, sdkFlags *sdkFlags) bool { + switch e.GetEventType() { + case enumspb.EVENT_TYPE_MARKER_RECORDED: + markerName := e.GetMarkerRecordedEventAttributes().GetMarkerName() + if markerName == versionMarkerName || markerName == mutableSideEffectMarkerName { + return true + } + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED: + return true + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW: + return true + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_FAILED: + return true + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CANCELED: + return true + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_TIMED_OUT: + return true + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_ADMITTED: + return true + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_ACCEPTED, + enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_REJECTED, + enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_UPDATE_COMPLETED: + protocolMsgCommandInUse := sdkFlags.tryUse(SDKFlagProtocolMessageCommand, false) + return !protocolMsgCommandInUse + } + return false +} + +// special check for upsert change version event +func skipDeterministicCheckForUpsertChangeVersion(events []*historypb.HistoryEvent, idx int) bool { + e := events[idx] + if e.GetEventType() == enumspb.EVENT_TYPE_MARKER_RECORDED && + e.GetMarkerRecordedEventAttributes().GetMarkerName() == versionMarkerName && + idx < len(events)-1 && + events[idx+1].GetEventType() == enumspb.EVENT_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES { + if _, ok := events[idx+1].GetUpsertWorkflowSearchAttributesEventAttributes().SearchAttributes.IndexedFields[TemporalChangeVersion]; ok { + return true + } + } + return false +} + +func matchReplayWithHistory( + replayCommands []*commandpb.Command, + historyEvents []*historypb.HistoryEvent, + msgs []outboxEntry, + sdkFlags *sdkFlags, +) error { + di := 0 + hi := 0 + hSize := len(historyEvents) + dSize := len(replayCommands) +matchLoop: + for hi < hSize || di < dSize { + var e *historypb.HistoryEvent + if hi < hSize { + e = historyEvents[hi] + if skipDeterministicCheckForUpsertChangeVersion(historyEvents, hi) { + hi += 2 + continue matchLoop + } + if skipDeterministicCheckForEvent(e, sdkFlags) { + hi++ + continue matchLoop + } + } + + var d *commandpb.Command + if di < dSize { + d = replayCommands[di] + if skipDeterministicCheckForCommand(d, sdkFlags) { + di++ + continue matchLoop + } + } + + if d == nil { + return historyMismatchErrorf("[TMPRL1100] nondeterministic workflow: missing replay command for %s", util.HistoryEventToString(e)) + } + + if e == nil { + return historyMismatchErrorf("[TMPRL1100] nondeterministic workflow: extra replay command for %s", util.CommandToString(d)) + } + + if !isCommandMatchEvent(d, e, msgs) { + return historyMismatchErrorf("[TMPRL1100] nondeterministic workflow: history event is %s, replay command is %s", + util.HistoryEventToString(e), util.CommandToString(d)) + } + + di++ + hi++ + } + return nil +} + +func lastPartOfName(name string) string { + lastDotIdx := strings.LastIndex(name, ".") + if lastDotIdx < 0 || lastDotIdx == len(name)-1 { + return name + } + return name[lastDotIdx+1:] +} + +func isCommandMatchEvent(d *commandpb.Command, e *historypb.HistoryEvent, obes []outboxEntry) bool { + switch d.GetCommandType() { + case enumspb.COMMAND_TYPE_PROTOCOL_MESSAGE: + msgid := d.GetProtocolMessageCommandAttributes().GetMessageId() + for _, entry := range obes { + if entry.msg.Id == msgid { + return entry.eventPredicate(e) + } + } + return false + + case enumspb.COMMAND_TYPE_SCHEDULE_ACTIVITY_TASK: + if e.GetEventType() != enumspb.EVENT_TYPE_ACTIVITY_TASK_SCHEDULED { + return false + } + eventAttributes := e.GetActivityTaskScheduledEventAttributes() + commandAttributes := d.GetScheduleActivityTaskCommandAttributes() + + if eventAttributes.GetActivityId() != commandAttributes.GetActivityId() || + lastPartOfName(eventAttributes.ActivityType.GetName()) != lastPartOfName(commandAttributes.ActivityType.GetName()) { + return false + } + + return true + + case enumspb.COMMAND_TYPE_REQUEST_CANCEL_ACTIVITY_TASK: + if e.GetEventType() != enumspb.EVENT_TYPE_ACTIVITY_TASK_CANCEL_REQUESTED { + return false + } + commandAttributes := d.GetRequestCancelActivityTaskCommandAttributes() + eventAttributes := e.GetActivityTaskCancelRequestedEventAttributes() + if eventAttributes.GetScheduledEventId() != commandAttributes.GetScheduledEventId() { + return false + } + + return true + + case enumspb.COMMAND_TYPE_START_TIMER: + if e.GetEventType() != enumspb.EVENT_TYPE_TIMER_STARTED { + return false + } + eventAttributes := e.GetTimerStartedEventAttributes() + commandAttributes := d.GetStartTimerCommandAttributes() + + if eventAttributes.GetTimerId() != commandAttributes.GetTimerId() { + return false + } + + return true + + case enumspb.COMMAND_TYPE_CANCEL_TIMER: + if e.GetEventType() != enumspb.EVENT_TYPE_TIMER_CANCELED { + return false + } + commandAttributes := d.GetCancelTimerCommandAttributes() + if e.GetEventType() == enumspb.EVENT_TYPE_TIMER_CANCELED { + eventAttributes := e.GetTimerCanceledEventAttributes() + if eventAttributes.GetTimerId() != commandAttributes.GetTimerId() { + return false + } + } + + return true + + case enumspb.COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION: + if e.GetEventType() != enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED { + return false + } + + return true + + case enumspb.COMMAND_TYPE_FAIL_WORKFLOW_EXECUTION: + if e.GetEventType() != enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_FAILED { + return false + } + + return true + + case enumspb.COMMAND_TYPE_RECORD_MARKER: + if e.GetEventType() != enumspb.EVENT_TYPE_MARKER_RECORDED { + return false + } + eventAttributes := e.GetMarkerRecordedEventAttributes() + commandAttributes := d.GetRecordMarkerCommandAttributes() + if eventAttributes.GetMarkerName() != commandAttributes.GetMarkerName() { + return false + } + + return true + + case enumspb.COMMAND_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION: + if e.GetEventType() != enumspb.EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED { + return false + } + eventAttributes := e.GetRequestCancelExternalWorkflowExecutionInitiatedEventAttributes() + commandAttributes := d.GetRequestCancelExternalWorkflowExecutionCommandAttributes() + if checkNamespacesInCommandAndEvent(eventAttributes.GetNamespace(), commandAttributes.GetNamespace()) || + eventAttributes.WorkflowExecution.GetWorkflowId() != commandAttributes.GetWorkflowId() { + return false + } + + return true + + case enumspb.COMMAND_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION: + if e.GetEventType() != enumspb.EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED { + return false + } + eventAttributes := e.GetSignalExternalWorkflowExecutionInitiatedEventAttributes() + commandAttributes := d.GetSignalExternalWorkflowExecutionCommandAttributes() + if checkNamespacesInCommandAndEvent(eventAttributes.GetNamespace(), commandAttributes.GetNamespace()) || + eventAttributes.GetSignalName() != commandAttributes.GetSignalName() || + eventAttributes.WorkflowExecution.GetWorkflowId() != commandAttributes.Execution.GetWorkflowId() { + return false + } + + return true + + case enumspb.COMMAND_TYPE_CANCEL_WORKFLOW_EXECUTION: + if e.GetEventType() != enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CANCELED { + return false + } + return true + + case enumspb.COMMAND_TYPE_CONTINUE_AS_NEW_WORKFLOW_EXECUTION: + if e.GetEventType() != enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW { + return false + } + + return true + + case enumspb.COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION: + if e.GetEventType() != enumspb.EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_INITIATED { + return false + } + eventAttributes := e.GetStartChildWorkflowExecutionInitiatedEventAttributes() + commandAttributes := d.GetStartChildWorkflowExecutionCommandAttributes() + if lastPartOfName(eventAttributes.WorkflowType.GetName()) != lastPartOfName(commandAttributes.WorkflowType.GetName()) { + return false + } + + return true + + case enumspb.COMMAND_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES: + if e.GetEventType() != enumspb.EVENT_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES { + return false + } + return true + + case enumspb.COMMAND_TYPE_MODIFY_WORKFLOW_PROPERTIES: + if e.GetEventType() != enumspb.EVENT_TYPE_WORKFLOW_PROPERTIES_MODIFIED { + return false + } + return true + + case enumspb.COMMAND_TYPE_SCHEDULE_NEXUS_OPERATION: + if e.GetEventType() != enumspb.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED { + return false + } + eventAttributes := e.GetNexusOperationScheduledEventAttributes() + commandAttributes := d.GetScheduleNexusOperationCommandAttributes() + + return eventAttributes.GetService() == commandAttributes.GetService() && + eventAttributes.GetOperation() == commandAttributes.GetOperation() + + case enumspb.COMMAND_TYPE_REQUEST_CANCEL_NEXUS_OPERATION: + if e.GetEventType() != enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUESTED { + return false + } + + eventAttributes := e.GetNexusOperationCancelRequestedEventAttributes() + commandAttributes := d.GetRequestCancelNexusOperationCommandAttributes() + + return eventAttributes.GetScheduledEventId() == commandAttributes.GetScheduledEventId() + } + + return false +} + +func isSearchAttributesMatched(attrFromEvent, attrFromCommand *commonpb.SearchAttributes) bool { + if attrFromEvent != nil && attrFromCommand != nil { + return reflect.DeepEqual(attrFromEvent.IndexedFields, attrFromCommand.IndexedFields) + } + return attrFromEvent == nil && attrFromCommand == nil +} + +func isMemoMatched(attrFromEvent, attrFromCommand *commonpb.Memo) bool { + if attrFromEvent != nil && attrFromCommand != nil { + return reflect.DeepEqual(attrFromEvent.Fields, attrFromCommand.Fields) + } + return attrFromEvent == nil && attrFromCommand == nil +} + +// return true if the check fails: +// +// namespace is not empty in command +// and namespace is not replayNamespace +// and namespaces unmatch in command and events +func checkNamespacesInCommandAndEvent(eventNamespace, commandNamespace string) bool { + if commandNamespace == "" || IsReplayNamespace(commandNamespace) { + return false + } + return eventNamespace != commandNamespace +} + +func (wth *workflowTaskHandlerImpl) completeWorkflow( + eventHandler *workflowExecutionEventHandlerImpl, + task *workflowservice.PollWorkflowTaskQueueResponse, + workflowContext *workflowExecutionContextImpl, + commands []*commandpb.Command, + messages []*protocolpb.Message, + forceNewWorkflowTask bool, +) interface{} { + // for query task + if task.Query != nil { + queryCompletedRequest := &workflowservice.RespondQueryTaskCompletedRequest{ + TaskToken: task.TaskToken, + Namespace: wth.namespace, + } + var panicErr *PanicError + if errors.As(workflowContext.err, &panicErr) { + queryCompletedRequest.CompletedType = enumspb.QUERY_RESULT_TYPE_FAILED + queryCompletedRequest.ErrorMessage = "Workflow panic: " + panicErr.Error() + return queryCompletedRequest + } + + result, err := eventHandler.ProcessQuery(task.Query.GetQueryType(), task.Query.QueryArgs, task.Query.Header) + if err != nil { + queryCompletedRequest.CompletedType = enumspb.QUERY_RESULT_TYPE_FAILED + queryCompletedRequest.ErrorMessage = err.Error() + } else { + queryCompletedRequest.CompletedType = enumspb.QUERY_RESULT_TYPE_ANSWERED + queryCompletedRequest.QueryResult = result + } + return queryCompletedRequest + } + + metricsHandler := wth.metricsHandler.WithTags(metrics.WorkflowTags( + eventHandler.workflowEnvironmentImpl.workflowInfo.WorkflowType.Name)) + + // complete workflow task + var closeCommand *commandpb.Command + var canceledErr *CanceledError + var contErr *ContinueAsNewError + + if errors.As(workflowContext.err, &canceledErr) { + // Workflow canceled + metricsHandler.Counter(metrics.WorkflowCanceledCounter).Inc(1) + closeCommand = createNewCommand(enumspb.COMMAND_TYPE_CANCEL_WORKFLOW_EXECUTION) + closeCommand.Attributes = &commandpb.Command_CancelWorkflowExecutionCommandAttributes{CancelWorkflowExecutionCommandAttributes: &commandpb.CancelWorkflowExecutionCommandAttributes{ + Details: convertErrDetailsToPayloads(canceledErr.details, wth.dataConverter), + }} + } else if errors.As(workflowContext.err, &contErr) { + // Continue as new error. + metricsHandler.Counter(metrics.WorkflowContinueAsNewCounter).Inc(1) + closeCommand = createNewCommand(enumspb.COMMAND_TYPE_CONTINUE_AS_NEW_WORKFLOW_EXECUTION) + + // ContinueAsNewError.RetryPolicy is optional. + // If not set, use the retry policy from the workflow context. + retryPolicy := contErr.RetryPolicy + if retryPolicy == nil { + retryPolicy = workflowContext.workflowInfo.RetryPolicy + } + + useCompat := determineInheritBuildIdFlagForCommand( + contErr.VersioningIntent, workflowContext.workflowInfo.TaskQueueName, contErr.TaskQueueName) + closeCommand.Attributes = &commandpb.Command_ContinueAsNewWorkflowExecutionCommandAttributes{ContinueAsNewWorkflowExecutionCommandAttributes: &commandpb.ContinueAsNewWorkflowExecutionCommandAttributes{ + WorkflowType: &commonpb.WorkflowType{Name: contErr.WorkflowType.Name}, + Input: contErr.Input, + TaskQueue: &taskqueuepb.TaskQueue{Name: contErr.TaskQueueName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, + WorkflowRunTimeout: durationpb.New(contErr.WorkflowRunTimeout), + WorkflowTaskTimeout: durationpb.New(contErr.WorkflowTaskTimeout), + Header: contErr.Header, + Memo: workflowContext.workflowInfo.Memo, + SearchAttributes: workflowContext.workflowInfo.SearchAttributes, + RetryPolicy: convertToPBRetryPolicy(retryPolicy), + InheritBuildId: useCompat, + }} + } else if workflowContext.err != nil { + // Workflow failures + metricsHandler.Counter(metrics.WorkflowFailedCounter).Inc(1) + closeCommand = createNewCommand(enumspb.COMMAND_TYPE_FAIL_WORKFLOW_EXECUTION) + failure := wth.failureConverter.ErrorToFailure(workflowContext.err) + closeCommand.Attributes = &commandpb.Command_FailWorkflowExecutionCommandAttributes{FailWorkflowExecutionCommandAttributes: &commandpb.FailWorkflowExecutionCommandAttributes{ + Failure: failure, + }} + } else if workflowContext.isWorkflowCompleted { + // Workflow completion + metricsHandler.Counter(metrics.WorkflowCompletedCounter).Inc(1) + closeCommand = createNewCommand(enumspb.COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION) + closeCommand.Attributes = &commandpb.Command_CompleteWorkflowExecutionCommandAttributes{CompleteWorkflowExecutionCommandAttributes: &commandpb.CompleteWorkflowExecutionCommandAttributes{ + Result: workflowContext.result, + }} + } + + if closeCommand != nil { + commands = append(commands, closeCommand) + elapsed := time.Since(workflowContext.workflowInfo.WorkflowStartTime) + metricsHandler.Timer(metrics.WorkflowEndToEndLatency).Record(elapsed) + forceNewWorkflowTask = false + } + + var queryResults map[string]*querypb.WorkflowQueryResult + if len(task.Queries) != 0 { + queryResults = make(map[string]*querypb.WorkflowQueryResult) + for queryID, query := range task.Queries { + result, err := eventHandler.ProcessQuery(query.GetQueryType(), query.QueryArgs, query.Header) + if err != nil { + queryResults[queryID] = &querypb.WorkflowQueryResult{ + ResultType: enumspb.QUERY_RESULT_TYPE_FAILED, + ErrorMessage: err.Error(), + } + } else { + queryResults[queryID] = &querypb.WorkflowQueryResult{ + ResultType: enumspb.QUERY_RESULT_TYPE_ANSWERED, + Answer: result, + } + } + } + } + + nonfirstLAAttempts := eventHandler.GatherLAAttemptsThisWFT() + + sdkFlags := eventHandler.sdkFlags.gatherNewSDKFlags() + langUsedFlags := make([]uint32, 0, len(sdkFlags)) + for _, flag := range sdkFlags { + langUsedFlags = append(langUsedFlags, uint32(flag)) + } + + builtRequest := &workflowservice.RespondWorkflowTaskCompletedRequest{ + TaskToken: task.TaskToken, + Commands: commands, + Messages: messages, + Identity: wth.identity, + ReturnNewWorkflowTask: true, + ForceCreateNewWorkflowTask: forceNewWorkflowTask, + BinaryChecksum: wth.workerBuildID, + QueryResults: queryResults, + Namespace: wth.namespace, + MeteringMetadata: &commonpb.MeteringMetadata{NonfirstLocalActivityExecutionAttempts: nonfirstLAAttempts}, + SdkMetadata: &sdk.WorkflowTaskCompletedMetadata{ + LangUsedFlags: langUsedFlags, + SdkName: eventHandler.getNewSdkNameAndReset(), + SdkVersion: eventHandler.getNewSdkVersionAndReset(), + }, + WorkerVersionStamp: &commonpb.WorkerVersionStamp{ + BuildId: wth.workerBuildID, + UseVersioning: wth.useBuildIDForVersioning, + }, + } + if wth.capabilities != nil && wth.capabilities.BuildIdBasedVersioning { + builtRequest.BinaryChecksum = "" + } + return builtRequest +} + +func (wth *workflowTaskHandlerImpl) executeAnyPressurePoints(event *historypb.HistoryEvent, isInReplay bool) error { + if wth.ppMgr != nil && !reflect.ValueOf(wth.ppMgr).IsNil() && !isInReplay { + switch event.GetEventType() { + case enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED: + return wth.ppMgr.Execute(pressurePointTypeWorkflowTaskStartTimeout) + case enumspb.EVENT_TYPE_ACTIVITY_TASK_SCHEDULED: + return wth.ppMgr.Execute(pressurePointTypeActivityTaskScheduleTimeout) + case enumspb.EVENT_TYPE_ACTIVITY_TASK_STARTED: + return wth.ppMgr.Execute(pressurePointTypeActivityTaskStartTimeout) + case enumspb.EVENT_TYPE_WORKFLOW_TASK_COMPLETED: + return wth.ppMgr.Execute(pressurePointTypeWorkflowTaskCompleted) + } + } + return nil +} + +func newActivityTaskHandler( + service workflowservice.WorkflowServiceClient, + params workerExecutionParameters, + registry *registry, +) ActivityTaskHandler { + return newActivityTaskHandlerWithCustomProvider(service, params, registry, nil) +} + +func newActivityTaskHandlerWithCustomProvider( + service workflowservice.WorkflowServiceClient, + params workerExecutionParameters, + registry *registry, + activityProvider activityProvider, +) ActivityTaskHandler { + return &activityTaskHandlerImpl{ + taskQueueName: params.TaskQueue, + identity: params.Identity, + service: service, + logger: params.Logger, + metricsHandler: params.MetricsHandler, + userContext: params.UserContext, + registry: registry, + activityProvider: activityProvider, + dataConverter: params.DataConverter, + failureConverter: params.FailureConverter, + workerStopCh: params.WorkerStopChannel, + contextPropagators: params.ContextPropagators, + namespace: params.Namespace, + defaultHeartbeatThrottleInterval: params.DefaultHeartbeatThrottleInterval, + maxHeartbeatThrottleInterval: params.MaxHeartbeatThrottleInterval, + versionStamp: &commonpb.WorkerVersionStamp{ + BuildId: params.getBuildID(), + UseVersioning: params.UseBuildIDForVersioning, + }, + } +} + +type temporalInvoker struct { + sync.Mutex + identity string + service workflowservice.WorkflowServiceClient + metricsHandler metrics.Handler + taskToken []byte + cancelHandler func() + // Amount of time to wait between each pending heartbeat send + heartbeatThrottleInterval time.Duration + hbBatchEndTimer *time.Timer // Whether we started a batch of operations that need to be reported in the cycle. This gets started on a user call. + lastDetailsToReport **commonpb.Payloads + closeCh chan struct{} + workerStopChannel <-chan struct{} + namespace string +} + +func (i *temporalInvoker) Heartbeat(ctx context.Context, details *commonpb.Payloads, skipBatching bool) error { + i.Lock() + defer i.Unlock() + + if i.hbBatchEndTimer != nil && !skipBatching { + // If we have started batching window, keep track of last reported progress. + i.lastDetailsToReport = &details + return nil + } + + isActivityCanceled, err := i.internalHeartBeat(ctx, details) + + // If the activity is canceled, the activity can ignore the cancellation and do its work + // and complete. Our cancellation is co-operative, so we will try to heartbeat. + if (err == nil || isActivityCanceled) && !skipBatching { + // We have successfully sent heartbeat, start next batching window. + i.lastDetailsToReport = nil + + // Create timer to fire before the threshold to report. + i.hbBatchEndTimer = time.NewTimer(i.heartbeatThrottleInterval) + + go func() { + select { + case <-i.hbBatchEndTimer.C: + // We are close to deadline. + case <-i.workerStopChannel: + // Activity worker is close to stop. This does the same steps as batch timer ends. + case <-i.closeCh: + // We got closed. + return + } + + // We close the batch and report the progress. + var detailsToReport **commonpb.Payloads + + i.Lock() + detailsToReport = i.lastDetailsToReport + i.hbBatchEndTimer.Stop() + i.hbBatchEndTimer = nil + i.Unlock() + + if detailsToReport != nil { + // TODO: there is a potential race condition here as the lock is released here and + // locked again in the Hearbeat() method. This possible that a heartbeat call from + // user activity grabs the lock first and calls internalHeartBeat before this + // batching goroutine, which means some activity progress will be lost. + _ = i.Heartbeat(ctx, *detailsToReport, false) + } + }() + } + + return err +} + +func (i *temporalInvoker) internalHeartBeat(ctx context.Context, details *commonpb.Payloads) (bool, error) { + isActivityCanceled := false + // We don't want the recording of the heartbeat to keep retrying the RPC + // longer than the throttle interval. However, sometimes the interval is so + // small that the context is cancelled before it even starts the call. + // Therefore, we'll make sure not to timeout the context faster than the + // minimum RPC timeout. + recordTimeout := i.heartbeatThrottleInterval + if recordTimeout < minRPCTimeout { + recordTimeout = minRPCTimeout + } + ctx, cancel := context.WithTimeout(ctx, recordTimeout) + defer cancel() + + err := recordActivityHeartbeat(ctx, i.service, i.metricsHandler, i.identity, i.taskToken, details) + + switch err.(type) { + case *CanceledError: + // We are asked to cancel. inform the activity about cancellation through context. + i.cancelHandler() + isActivityCanceled = true + + case *serviceerror.NotFound, *serviceerror.NamespaceNotActive, *serviceerror.NamespaceNotFound: + // We will pass these through as cancellation for now but something we can change + // later when we have setter on cancel handler. + i.cancelHandler() + isActivityCanceled = true + case nil: + // No error, do nothing. + default: + // Transient errors are getting retried for the duration of the heartbeat timeout. + // The fact that error has been returned means that activity should now be timed out, hence we should + // propagate cancellation to the handler. + err, _ := status.FromError(err) + if retry.IsStatusCodeRetryable(err) { + i.cancelHandler() + isActivityCanceled = true + } + } + + if err != nil { + logger := GetActivityLogger(ctx) + logger.Warn("RecordActivityHeartbeat with error", tagError, err) + } + + // This error won't be returned to user check RecordActivityHeartbeat(). + return isActivityCanceled, err +} + +func (i *temporalInvoker) Close(ctx context.Context, flushBufferedHeartbeat bool) { + i.Lock() + defer i.Unlock() + close(i.closeCh) + if i.hbBatchEndTimer != nil { + i.hbBatchEndTimer.Stop() + if flushBufferedHeartbeat && i.lastDetailsToReport != nil { + _, _ = i.internalHeartBeat(ctx, *i.lastDetailsToReport) + i.lastDetailsToReport = nil + } + } +} + +func (i *temporalInvoker) GetClient(options ClientOptions) Client { + return NewServiceClient(i.service, nil, options) +} + +func newServiceInvoker( + taskToken []byte, + identity string, + service workflowservice.WorkflowServiceClient, + metricsHandler metrics.Handler, + cancelHandler func(), + heartbeatThrottleInterval time.Duration, + workerStopChannel <-chan struct{}, + namespace string, +) ServiceInvoker { + return &temporalInvoker{ + taskToken: taskToken, + identity: identity, + service: service, + metricsHandler: metricsHandler, + cancelHandler: cancelHandler, + heartbeatThrottleInterval: heartbeatThrottleInterval, + closeCh: make(chan struct{}), + workerStopChannel: workerStopChannel, + namespace: namespace, + } +} + +// Execute executes an implementation of the activity. +func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice.PollActivityTaskQueueResponse) (result interface{}, err error) { + traceLog(func() { + ath.logger.Debug("Processing new activity task", + tagWorkflowID, t.WorkflowExecution.GetWorkflowId(), + tagRunID, t.WorkflowExecution.GetRunId(), + tagActivityType, t.ActivityType.GetName(), + tagAttempt, t.Attempt, + ) + }) + + rootCtx := ath.userContext + if rootCtx == nil { + rootCtx = context.Background() + } + canCtx, cancel := context.WithCancel(rootCtx) + defer cancel() + + heartbeatThrottleInterval := ath.getHeartbeatThrottleInterval(t.GetHeartbeatTimeout().AsDuration()) + invoker := newServiceInvoker( + t.TaskToken, ath.identity, ath.service, ath.metricsHandler, cancel, heartbeatThrottleInterval, + ath.workerStopCh, ath.namespace) + + workflowType := t.WorkflowType.GetName() + activityType := t.ActivityType.GetName() + metricsHandler := ath.metricsHandler.WithTags(metrics.ActivityTags(workflowType, activityType, ath.taskQueueName)) + ctx, err := WithActivityTask(canCtx, t, taskQueue, invoker, ath.logger, metricsHandler, + ath.dataConverter, ath.workerStopCh, ath.contextPropagators, ath.registry.interceptors) + if err != nil { + return nil, err + } + + // We must capture the context here because it is changed later to one that is + // cancelled when the activity is done + defer func(ctx context.Context) { + _, activityCompleted := result.(*workflowservice.RespondActivityTaskCompletedRequest) + invoker.Close(ctx, !activityCompleted) // flush buffered heartbeat if activity was not successfully completed. + }(ctx) + + activityImplementation := ath.getActivity(activityType) + if activityImplementation == nil { + // In case if activity is not registered we should report a failure to the server to allow activity retry + // instead of making it stuck on the same attempt. + metricsHandler.Counter(metrics.UnregisteredActivityInvocationCounter).Inc(1) + return convertActivityResultToRespondRequest(ath.identity, t.TaskToken, nil, + NewActivityNotRegisteredError(activityType, ath.getRegisteredActivityNames()), + ath.dataConverter, ath.failureConverter, ath.namespace, false, ath.versionStamp), nil + } + + // panic handler + defer func() { + if p := recover(); p != nil { + topLine := fmt.Sprintf("activity for %s [panic]:", ath.taskQueueName) + st := getStackTraceRaw(topLine, 7, 0) + ath.logger.Error("Activity panic.", + tagWorkflowID, t.WorkflowExecution.GetWorkflowId(), + tagRunID, t.WorkflowExecution.GetRunId(), + tagActivityType, activityType, + tagAttempt, t.Attempt, + tagPanicError, fmt.Sprintf("%v", p), + tagPanicStack, st) + metricsHandler.Counter(metrics.ActivityTaskErrorCounter).Inc(1) + panicErr := newPanicError(p, st) + result = convertActivityResultToRespondRequest(ath.identity, t.TaskToken, nil, panicErr, + ath.dataConverter, ath.failureConverter, ath.namespace, false, ath.versionStamp) + } + }() + + // propagate context information into the activity context from the headers + ctx, err = contextWithHeaderPropagated(ctx, t.Header, ath.contextPropagators) + if err != nil { + return nil, err + } + + info := getActivityEnv(ctx) + ctx, dlCancelFunc := context.WithDeadline(ctx, info.deadline) + defer dlCancelFunc() + + output, err := activityImplementation.Execute(ctx, t.Input) + // Check if context canceled at a higher level before we cancel it ourselves + isActivityCancel := ctx.Err() == context.Canceled + + dlCancelFunc() + if <-ctx.Done(); ctx.Err() == context.DeadlineExceeded { + ath.logger.Info("Activity complete after timeout.", + tagWorkflowID, t.WorkflowExecution.GetWorkflowId(), + tagRunID, t.WorkflowExecution.GetRunId(), + tagActivityType, activityType, + tagAttempt, t.Attempt, + tagResult, output, + tagError, err, + ) + return nil, ctx.Err() + } + if err != nil && err != ErrActivityResultPending { + ath.logger.Error("Activity error.", + tagWorkflowID, t.WorkflowExecution.GetWorkflowId(), + tagRunID, t.WorkflowExecution.GetRunId(), + tagActivityType, activityType, + tagAttempt, t.Attempt, + tagError, err, + ) + } + return convertActivityResultToRespondRequest(ath.identity, t.TaskToken, output, err, + ath.dataConverter, ath.failureConverter, ath.namespace, isActivityCancel, ath.versionStamp), nil +} + +func (ath *activityTaskHandlerImpl) getActivity(name string) activity { + if ath.activityProvider != nil { + return ath.activityProvider(name) + } + + if a, ok := ath.registry.GetActivity(name); ok { + return a + } + + return nil +} + +func (ath *activityTaskHandlerImpl) getRegisteredActivityNames() (activityNames []string) { + for _, a := range ath.registry.getRegisteredActivities() { + activityNames = append(activityNames, a.ActivityType().Name) + } + return +} + +func (ath *activityTaskHandlerImpl) getHeartbeatThrottleInterval(heartbeatTimeout time.Duration) time.Duration { + // Set interval as 80% of timeout if present, or the configured default if + // present, or the system default otherwise + var heartbeatThrottleInterval time.Duration + if heartbeatTimeout > 0 { + heartbeatThrottleInterval = time.Duration(0.8 * float64(heartbeatTimeout)) + } else if ath.defaultHeartbeatThrottleInterval > 0 { + heartbeatThrottleInterval = ath.defaultHeartbeatThrottleInterval + } else { + heartbeatThrottleInterval = defaultDefaultHeartbeatThrottleInterval + } + + // Use the configured max if present, or the system default otherwise + maxHeartbeatThrottleInterval := ath.maxHeartbeatThrottleInterval + if maxHeartbeatThrottleInterval == 0 { + maxHeartbeatThrottleInterval = defaultMaxHeartbeatThrottleInterval + } + + // Limit interval to a max + if heartbeatThrottleInterval > maxHeartbeatThrottleInterval { + heartbeatThrottleInterval = maxHeartbeatThrottleInterval + } + return heartbeatThrottleInterval +} + +func createNewCommand(commandType enumspb.CommandType) *commandpb.Command { + return &commandpb.Command{ + CommandType: commandType, + } +} + +func recordActivityHeartbeat(ctx context.Context, service workflowservice.WorkflowServiceClient, metricsHandler metrics.Handler, + identity string, taskToken []byte, details *commonpb.Payloads, +) error { + namespace := getNamespaceFromActivityCtx(ctx) + request := &workflowservice.RecordActivityTaskHeartbeatRequest{ + TaskToken: taskToken, + Details: details, + Identity: identity, + Namespace: namespace, + } + + var heartbeatResponse *workflowservice.RecordActivityTaskHeartbeatResponse + grpcCtx, cancel := newGRPCContext(ctx, + grpcMetricsHandler(metricsHandler), + defaultGrpcRetryParameters(ctx)) + defer cancel() + + heartbeatResponse, err := service.RecordActivityTaskHeartbeat(grpcCtx, request) + if err == nil && heartbeatResponse != nil && heartbeatResponse.GetCancelRequested() { + return NewCanceledError() + } + return err +} + +func recordActivityHeartbeatByID(ctx context.Context, service workflowservice.WorkflowServiceClient, metricsHandler metrics.Handler, + identity, namespace, workflowID, runID, activityID string, details *commonpb.Payloads, +) error { + request := &workflowservice.RecordActivityTaskHeartbeatByIdRequest{ + Namespace: namespace, + WorkflowId: workflowID, + RunId: runID, + ActivityId: activityID, + Details: details, + Identity: identity, + } + + var heartbeatResponse *workflowservice.RecordActivityTaskHeartbeatByIdResponse + grpcCtx, cancel := newGRPCContext(ctx, + grpcMetricsHandler(metricsHandler), + defaultGrpcRetryParameters(ctx)) + defer cancel() + + heartbeatResponse, err := service.RecordActivityTaskHeartbeatById(grpcCtx, request) + if err == nil && heartbeatResponse != nil && heartbeatResponse.GetCancelRequested() { + return NewCanceledError() + } + return err +} + +// This enables verbose logging in the client library. +// check worker.EnableVerboseLogging() +func traceLog(fn func()) { + if enableVerboseLogging { + fn() + } +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_task_pollers.go b/vendor/go.temporal.io/sdk/internal/internal_task_pollers.go new file mode 100644 index 00000000000..da8c7c70a19 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_task_pollers.go @@ -0,0 +1,1260 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +// All code in this file is private to the package. + +import ( + "context" + "errors" + "fmt" + "sync" + "time" + + "google.golang.org/protobuf/types/known/durationpb" + "google.golang.org/protobuf/types/known/wrapperspb" + + "github.com/pborman/uuid" + + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + historypb "go.temporal.io/api/history/v1" + taskqueuepb "go.temporal.io/api/taskqueue/v1" + "go.temporal.io/api/workflowservice/v1" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/internal/common/serializer" + "go.temporal.io/sdk/log" +) + +const ( + // Server returns empty task after dynamicconfig.MatchingLongPollExpirationInterval (default is 60 seconds). + // pollTaskServiceTimeOut should be dynamicconfig.MatchingLongPollExpirationInterval + some delta for full round trip to matching + // because empty task should be returned before timeout is expired (expired timeout counts against SLO). + pollTaskServiceTimeOut = 70 * time.Second + + stickyWorkflowTaskScheduleToStartTimeoutSeconds = 5 + + ratioToForceCompleteWorkflowTaskComplete = 0.8 +) + +type ( + // taskPoller interface to poll and process for task + taskPoller interface { + // PollTask polls for one new task + PollTask() (taskForWorker, error) + // ProcessTask processes a task + ProcessTask(interface{}) error + } + + taskForWorker interface { + isEmpty() bool + } + + // basePoller is the base class for all poller implementations + basePoller struct { + metricsHandler metrics.Handler // base metric handler used for rpc calls + stopC <-chan struct{} + // The worker's build ID, either as defined by the user or automatically set + workerBuildID string + // Whether the worker has opted in to the build-id based versioning feature + useBuildIDVersioning bool + // Server's capabilities + capabilities *workflowservice.GetSystemInfoResponse_Capabilities + } + + // numPollerMetric tracks the number of active pollers and publishes a metric on it. + numPollerMetric struct { + lock sync.Mutex + numPollers int32 + gauge metrics.Gauge + } + + // workflowTaskPoller implements polling/processing a workflow task + workflowTaskPoller struct { + basePoller + namespace string + taskQueueName string + identity string + service workflowservice.WorkflowServiceClient + taskHandler WorkflowTaskHandler + contextManager WorkflowContextManager + logger log.Logger + dataConverter converter.DataConverter + failureConverter converter.FailureConverter + + stickyUUID string + StickyScheduleToStartTimeout time.Duration + + pendingRegularPollCount int + pendingStickyPollCount int + stickyBacklog int64 + requestLock sync.Mutex + stickyCacheSize int + eagerActivityExecutor *eagerActivityExecutor + + numNormalPollerMetric *numPollerMetric + numStickyPollerMetric *numPollerMetric + } + + // activityTaskPoller implements polling/processing a workflow task + activityTaskPoller struct { + basePoller + namespace string + taskQueueName string + identity string + service workflowservice.WorkflowServiceClient + taskHandler ActivityTaskHandler + logger log.Logger + activitiesPerSecond float64 + numPollerMetric *numPollerMetric + } + + historyIteratorImpl struct { + iteratorFunc func(nextPageToken []byte) (*historypb.History, []byte, error) + execution *commonpb.WorkflowExecution + nextPageToken []byte + namespace string + service workflowservice.WorkflowServiceClient + maxEventID int64 + metricsHandler metrics.Handler + taskQueue string + } + + localActivityTaskPoller struct { + basePoller + handler *localActivityTaskHandler + logger log.Logger + laTunnel *localActivityTunnel + } + + localActivityTaskHandler struct { + userContext context.Context + metricsHandler metrics.Handler + logger log.Logger + dataConverter converter.DataConverter + contextPropagators []ContextPropagator + interceptors []WorkerInterceptor + } + + localActivityResult struct { + result *commonpb.Payloads + err error + task *localActivityTask + backoff time.Duration + } + + localActivityTunnel struct { + taskCh chan *localActivityTask + resultCh chan eagerOrPolledTask + stopCh <-chan struct{} + } +) + +func newNumPollerMetric(metricsHandler metrics.Handler, pollerType string) *numPollerMetric { + return &numPollerMetric{ + gauge: metricsHandler.WithTags(metrics.PollerTags(pollerType)).Gauge(metrics.NumPoller), + } +} + +func (npm *numPollerMetric) increment() { + npm.lock.Lock() + defer npm.lock.Unlock() + npm.numPollers += 1 + npm.gauge.Update(float64(npm.numPollers)) +} + +func (npm *numPollerMetric) decrement() { + npm.lock.Lock() + defer npm.lock.Unlock() + npm.numPollers -= 1 + npm.gauge.Update(float64(npm.numPollers)) +} + +func newLocalActivityTunnel(stopCh <-chan struct{}) *localActivityTunnel { + return &localActivityTunnel{ + taskCh: make(chan *localActivityTask, 100000), + resultCh: make(chan eagerOrPolledTask), + stopCh: stopCh, + } +} + +func (lat *localActivityTunnel) getTask() *localActivityTask { + select { + case task := <-lat.taskCh: + return task + case <-lat.stopCh: + return nil + } +} + +func (lat *localActivityTunnel) sendTask(task *localActivityTask) bool { + select { + case lat.taskCh <- task: + return true + case <-lat.stopCh: + return false + } +} + +func isClientSideError(err error) bool { + // If an activity execution exceeds deadline. + return err == context.DeadlineExceeded +} + +// stopping returns true if worker is stopping right now +func (bp *basePoller) stopping() bool { + select { + case <-bp.stopC: + return true + default: + return false + } +} + +// doPoll runs the given pollFunc in a separate go routine. Returns when either of the conditions are met: +// - poll succeeds, poll fails or worker is stopping +func (bp *basePoller) doPoll(pollFunc func(ctx context.Context) (taskForWorker, error)) (taskForWorker, error) { + if bp.stopping() { + return nil, errStop + } + + var err error + var result taskForWorker + + doneC := make(chan struct{}) + ctx, cancel := newGRPCContext(context.Background(), grpcTimeout(pollTaskServiceTimeOut), grpcLongPoll(true)) + + go func() { + result, err = pollFunc(ctx) + cancel() + close(doneC) + }() + + select { + case <-doneC: + return result, err + case <-bp.stopC: + cancel() + return nil, errStop + } +} + +func (bp *basePoller) getCapabilities() *workflowservice.GetSystemInfoResponse_Capabilities { + if bp.capabilities == nil { + return &workflowservice.GetSystemInfoResponse_Capabilities{} + } + return bp.capabilities +} + +// newWorkflowTaskPoller creates a new workflow task poller which must have a one to one relationship to workflow worker +func newWorkflowTaskPoller( + taskHandler WorkflowTaskHandler, + contextManager WorkflowContextManager, + service workflowservice.WorkflowServiceClient, + params workerExecutionParameters, +) *workflowTaskPoller { + return &workflowTaskPoller{ + basePoller: basePoller{ + metricsHandler: params.MetricsHandler, + stopC: params.WorkerStopChannel, + workerBuildID: params.getBuildID(), + useBuildIDVersioning: params.UseBuildIDForVersioning, + capabilities: params.capabilities, + }, + service: service, + namespace: params.Namespace, + taskQueueName: params.TaskQueue, + identity: params.Identity, + taskHandler: taskHandler, + contextManager: contextManager, + logger: params.Logger, + dataConverter: params.DataConverter, + failureConverter: params.FailureConverter, + stickyUUID: uuid.New(), + StickyScheduleToStartTimeout: params.StickyScheduleToStartTimeout, + stickyCacheSize: params.cache.MaxWorkflowCacheSize(), + eagerActivityExecutor: params.eagerActivityExecutor, + numNormalPollerMetric: newNumPollerMetric(params.MetricsHandler, metrics.PollerTypeWorkflowTask), + numStickyPollerMetric: newNumPollerMetric(params.MetricsHandler, metrics.PollerTypeWorkflowStickyTask), + } +} + +// PollTask polls a new task +func (wtp *workflowTaskPoller) PollTask() (taskForWorker, error) { + // Get the task. + workflowTask, err := wtp.doPoll(wtp.poll) + if err != nil { + return nil, err + } + + return workflowTask, nil +} + +// ProcessTask processes a task which could be workflow task or local activity result +func (wtp *workflowTaskPoller) ProcessTask(task interface{}) error { + if wtp.stopping() { + return errStop + } + + switch task := task.(type) { + case *workflowTask: + return wtp.processWorkflowTask(task) + case *eagerWorkflowTask: + return wtp.processWorkflowTask(wtp.toWorkflowTask(task.task)) + default: + panic("unknown task type.") + } +} + +func (wtp *workflowTaskPoller) processWorkflowTask(task *workflowTask) error { + if task.task == nil { + // We didn't have task, poll might have timeout. + traceLog(func() { + wtp.logger.Debug("Workflow task unavailable") + }) + return nil + } + + doneCh := make(chan struct{}) + laResultCh := make(chan *localActivityResult) + laRetryCh := make(chan *localActivityTask) + // close doneCh so local activity worker won't get blocked forever when trying to send back result to laResultCh. + defer close(doneCh) + + wfctx, err := wtp.contextManager.GetOrCreateWorkflowContext(task.task, task.historyIterator) + if err != nil { + return err + } + var taskErr error + defer func() { + wfctx.Unlock(taskErr) + }() + + for { + startTime := time.Now() + task.doneCh = doneCh + task.laResultCh = laResultCh + task.laRetryCh = laRetryCh + var completedRequest interface{} + completedRequest, taskErr = wtp.taskHandler.ProcessWorkflowTask( + task, + wfctx, + func(response interface{}, startTime time.Time) (*workflowTask, error) { + wtp.logger.Debug("Force RespondWorkflowTaskCompleted.", "TaskStartedEventID", task.task.GetStartedEventId()) + heartbeatResponse, err := wtp.RespondTaskCompletedWithMetrics(response, nil, task.task, startTime) + if err != nil { + return nil, err + } + if heartbeatResponse == nil || heartbeatResponse.WorkflowTask == nil { + return nil, nil + } + task := wtp.toWorkflowTask(heartbeatResponse.WorkflowTask) + task.doneCh = doneCh + task.laResultCh = laResultCh + task.laRetryCh = laRetryCh + return task, nil + }, + ) + if completedRequest == nil && taskErr == nil { + return nil + } + if _, ok := taskErr.(workflowTaskHeartbeatError); ok { + return taskErr + } + response, err := wtp.RespondTaskCompletedWithMetrics(completedRequest, taskErr, task.task, startTime) + if err != nil { + // If we get an error responding to the workflow task we need to evict the execution from the cache. + taskErr = err + return err + } + + if eventLevel := response.GetResetHistoryEventId(); eventLevel != 0 { + wfctx.SetPreviousStartedEventID(eventLevel) + } + + if response == nil || response.WorkflowTask == nil || taskErr != nil { + return nil + } + + // we are getting new workflow task, so reset the workflowTask and continue process the new one + task = wtp.toWorkflowTask(response.WorkflowTask) + } +} + +func (wtp *workflowTaskPoller) RespondTaskCompletedWithMetrics( + completedRequest interface{}, + taskErr error, + task *workflowservice.PollWorkflowTaskQueueResponse, + startTime time.Time, +) (response *workflowservice.RespondWorkflowTaskCompletedResponse, err error) { + metricsHandler := wtp.metricsHandler.WithTags(metrics.WorkflowTags(task.WorkflowType.GetName())) + if taskErr != nil { + wtp.logger.Warn("Failed to process workflow task.", + tagWorkflowType, task.WorkflowType.GetName(), + tagWorkflowID, task.WorkflowExecution.GetWorkflowId(), + tagRunID, task.WorkflowExecution.GetRunId(), + tagAttempt, task.Attempt, + tagError, taskErr) + failWorkflowTask := wtp.errorToFailWorkflowTask(task.TaskToken, taskErr) + failureReason := "WorkflowError" + if failWorkflowTask.Cause == enumspb.WORKFLOW_TASK_FAILED_CAUSE_NON_DETERMINISTIC_ERROR { + failureReason = "NonDeterminismError" + } + metricsHandler.WithTags(metrics.WorkflowTaskFailedTags(failureReason)).Counter(metrics.WorkflowTaskExecutionFailureCounter).Inc(1) + completedRequest = failWorkflowTask + } + + metricsHandler.Timer(metrics.WorkflowTaskExecutionLatency).Record(time.Since(startTime)) + + response, err = wtp.RespondTaskCompleted(completedRequest, task) + return +} + +func (wtp *workflowTaskPoller) RespondTaskCompleted( + completedRequest interface{}, + task *workflowservice.PollWorkflowTaskQueueResponse, +) (response *workflowservice.RespondWorkflowTaskCompletedResponse, err error) { + ctx := context.Background() + // Respond task completion. + grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler( + wtp.metricsHandler.WithTags(metrics.RPCTags(task.GetWorkflowType().GetName(), + metrics.NoneTagValue, metrics.NoneTagValue))), + defaultGrpcRetryParameters(ctx)) + defer cancel() + switch request := completedRequest.(type) { + case *workflowservice.RespondWorkflowTaskFailedRequest: + // Only fail workflow task on first attempt, subsequent failure on the same workflow task will timeout. + // This is to avoid spin on the failed workflow task. Checking Attempt not nil for older server. + if task.GetAttempt() == 1 { + _, err = wtp.service.RespondWorkflowTaskFailed(grpcCtx, request) + if err != nil { + traceLog(func() { + wtp.logger.Debug("RespondWorkflowTaskFailed failed.", tagError, err) + }) + } + } + case *workflowservice.RespondWorkflowTaskCompletedRequest: + if request.StickyAttributes == nil && wtp.stickyCacheSize > 0 { + request.StickyAttributes = &taskqueuepb.StickyExecutionAttributes{ + WorkerTaskQueue: &taskqueuepb.TaskQueue{ + Name: getWorkerTaskQueue(wtp.stickyUUID), + Kind: enumspb.TASK_QUEUE_KIND_STICKY, + NormalName: wtp.taskQueueName, + }, + ScheduleToStartTimeout: durationpb.New(wtp.StickyScheduleToStartTimeout), + } + } + eagerReserved := wtp.eagerActivityExecutor.applyToRequest(request) + response, err = wtp.service.RespondWorkflowTaskCompleted(grpcCtx, request) + if err != nil { + traceLog(func() { + wtp.logger.Debug("RespondWorkflowTaskCompleted failed.", tagError, err) + }) + } + wtp.eagerActivityExecutor.handleResponse(response, eagerReserved) + case *workflowservice.RespondQueryTaskCompletedRequest: + _, err = wtp.service.RespondQueryTaskCompleted(grpcCtx, request) + if err != nil { + traceLog(func() { + wtp.logger.Debug("RespondQueryTaskCompleted failed.", tagError, err) + }) + } + default: + // should not happen + panic("unknown request type from ProcessWorkflowTask()") + } + return +} + +func (wtp *workflowTaskPoller) errorToFailWorkflowTask(taskToken []byte, err error) *workflowservice.RespondWorkflowTaskFailedRequest { + cause := enumspb.WORKFLOW_TASK_FAILED_CAUSE_WORKFLOW_WORKER_UNHANDLED_FAILURE + // If it was a panic due to a bad state machine or if it was a history + // mismatch error, mark as non-deterministic + if panicErr, _ := err.(*workflowPanicError); panicErr != nil { + if _, badStateMachine := panicErr.value.(stateMachineIllegalStatePanic); badStateMachine { + cause = enumspb.WORKFLOW_TASK_FAILED_CAUSE_NON_DETERMINISTIC_ERROR + } + } else if _, mismatch := err.(historyMismatchError); mismatch { + cause = enumspb.WORKFLOW_TASK_FAILED_CAUSE_NON_DETERMINISTIC_ERROR + } else if _, unknown := err.(unknownSdkFlagError); unknown { + cause = enumspb.WORKFLOW_TASK_FAILED_CAUSE_NON_DETERMINISTIC_ERROR + } + + builtRequest := &workflowservice.RespondWorkflowTaskFailedRequest{ + TaskToken: taskToken, + Cause: cause, + Failure: wtp.failureConverter.ErrorToFailure(err), + Identity: wtp.identity, + BinaryChecksum: wtp.workerBuildID, + Namespace: wtp.namespace, + WorkerVersion: &commonpb.WorkerVersionStamp{ + BuildId: wtp.workerBuildID, + UseVersioning: wtp.useBuildIDVersioning, + }, + } + + if wtp.getCapabilities().BuildIdBasedVersioning { + builtRequest.BinaryChecksum = "" + } + + return builtRequest +} + +func newLocalActivityPoller( + params workerExecutionParameters, + laTunnel *localActivityTunnel, + interceptors []WorkerInterceptor, +) *localActivityTaskPoller { + handler := &localActivityTaskHandler{ + userContext: params.UserContext, + metricsHandler: params.MetricsHandler, + logger: params.Logger, + dataConverter: params.DataConverter, + contextPropagators: params.ContextPropagators, + interceptors: interceptors, + } + return &localActivityTaskPoller{ + basePoller: basePoller{metricsHandler: params.MetricsHandler, stopC: params.WorkerStopChannel}, + handler: handler, + logger: params.Logger, + laTunnel: laTunnel, + } +} + +func (latp *localActivityTaskPoller) PollTask() (taskForWorker, error) { + return latp.laTunnel.getTask(), nil +} + +func (latp *localActivityTaskPoller) ProcessTask(task interface{}) error { + if latp.stopping() { + return errStop + } + + result := latp.handler.executeLocalActivityTask(task.(*localActivityTask)) + // We need to send back the local activity result to unblock workflowTaskPoller.processWorkflowTask() which is + // synchronously listening on the laResultCh. We also want to make sure we don't block here forever in case + // processWorkflowTask() already returns and nobody is receiving from laResultCh. We guarantee that doneCh is closed + // before returning from workflowTaskPoller.processWorkflowTask(). + select { + case result.task.workflowTask.laResultCh <- result: + return nil + case <-result.task.workflowTask.doneCh: + // processWorkflowTask() already returns, just drop this local activity result. + return nil + } +} + +func (lath *localActivityTaskHandler) executeLocalActivityTask(task *localActivityTask) (result *localActivityResult) { + workflowType := task.params.WorkflowInfo.WorkflowType.Name + activityType := task.params.ActivityType + metricsHandler := lath.metricsHandler.WithTags(metrics.LocalActivityTags(workflowType, activityType)) + + metricsHandler.Counter(metrics.LocalActivityTotalCounter).Inc(1) + + ae := activityExecutor{name: activityType, fn: task.params.ActivityFn} + traceLog(func() { + lath.logger.Debug("Processing new local activity task", + tagWorkflowID, task.params.WorkflowInfo.WorkflowExecution.ID, + tagRunID, task.params.WorkflowInfo.WorkflowExecution.RunID, + tagActivityType, activityType, + tagAttempt, task.attempt, + ) + }) + ctx, err := WithLocalActivityTask(lath.userContext, task, lath.logger, lath.metricsHandler, + lath.dataConverter, lath.interceptors) + if err != nil { + return &localActivityResult{task: task, err: fmt.Errorf("failed building context: %w", err)} + } + + // propagate context information into the local activity context from the headers + ctx, err = contextWithHeaderPropagated(ctx, task.header, lath.contextPropagators) + if err != nil { + return &localActivityResult{task: task, err: err} + } + + info := getActivityEnv(ctx) + ctx, cancel := context.WithDeadline(ctx, info.deadline) + defer cancel() + + task.Lock() + if task.canceled { + task.Unlock() + return &localActivityResult{err: ErrCanceled, task: task} + } + task.attemptsThisWFT += 1 + task.cancelFunc = cancel + task.Unlock() + + var laResult *commonpb.Payloads + doneCh := make(chan struct{}) + go func(ch chan struct{}) { + laStartTime := time.Now() + defer close(ch) + + // panic handler + defer func() { + if p := recover(); p != nil { + topLine := fmt.Sprintf("local activity for %s [panic]:", activityType) + st := getStackTraceRaw(topLine, 7, 0) + lath.logger.Error("LocalActivity panic.", + tagWorkflowID, task.params.WorkflowInfo.WorkflowExecution.ID, + tagRunID, task.params.WorkflowInfo.WorkflowExecution.RunID, + tagActivityType, activityType, + tagAttempt, task.attempt, + tagPanicError, fmt.Sprintf("%v", p), + tagPanicStack, st) + metricsHandler.Counter(metrics.LocalActivityErrorCounter).Inc(1) + err = newPanicError(p, st) + } + if err != nil { + metricsHandler.Counter(metrics.LocalActivityFailedCounter).Inc(1) + metricsHandler.Counter(metrics.LocalActivityExecutionFailedCounter).Inc(1) + } + }() + + laResult, err = ae.ExecuteWithActualArgs(ctx, task.params.InputArgs) + executionLatency := time.Since(laStartTime) + metricsHandler.Timer(metrics.LocalActivityExecutionLatency).Record(executionLatency) + if time.Now().After(info.deadline) { + // If local activity takes longer than expected timeout, the context would already be DeadlineExceeded and + // the result would be discarded. Print a warning in this case. + lath.logger.Warn("LocalActivity takes too long to complete.", + "LocalActivityID", task.activityID, + "LocalActivityType", activityType, + "ScheduleToCloseTimeout", task.params.ScheduleToCloseTimeout, + "StartToCloseTimeout", task.params.StartToCloseTimeout, + "ActualExecutionDuration", executionLatency) + } + }(doneCh) + +WaitResult: + select { + case <-ctx.Done(): + select { + case <-doneCh: + // double check if result is ready. + break WaitResult + default: + } + + // context is done + if ctx.Err() == context.Canceled { + metricsHandler.Counter(metrics.LocalActivityCanceledCounter).Inc(1) + metricsHandler.Counter(metrics.LocalActivityExecutionCanceledCounter).Inc(1) + return &localActivityResult{err: ErrCanceled, task: task} + } else if ctx.Err() == context.DeadlineExceeded { + if task.params.ScheduleToCloseTimeout != 0 && time.Now().After(info.scheduledTime.Add(task.params.ScheduleToCloseTimeout)) { + return &localActivityResult{err: ErrDeadlineExceeded, task: task} + } else { + return &localActivityResult{err: NewTimeoutError("deadline exceeded", enumspb.TIMEOUT_TYPE_START_TO_CLOSE, nil), task: task} + } + } else { + // should not happen + return &localActivityResult{err: NewApplicationError("unexpected context done", "", true, nil), task: task} + } + case <-doneCh: + // local activity completed + } + + if err == nil { + metricsHandler. + Timer(metrics.LocalActivitySucceedEndToEndLatency). + Record(time.Since(task.params.ScheduledTime)) + } + return &localActivityResult{result: laResult, err: err, task: task} +} + +func (wtp *workflowTaskPoller) release(kind enumspb.TaskQueueKind) { + if wtp.stickyCacheSize <= 0 { + return + } + + wtp.requestLock.Lock() + if kind == enumspb.TASK_QUEUE_KIND_STICKY { + wtp.pendingStickyPollCount-- + } else { + wtp.pendingRegularPollCount-- + } + wtp.requestLock.Unlock() +} + +func (wtp *workflowTaskPoller) updateBacklog(taskQueueKind enumspb.TaskQueueKind, backlogCountHint int64) { + if taskQueueKind == enumspb.TASK_QUEUE_KIND_NORMAL || wtp.stickyCacheSize <= 0 { + // we only care about sticky backlog for now. + return + } + wtp.requestLock.Lock() + wtp.stickyBacklog = backlogCountHint + wtp.requestLock.Unlock() +} + +// getNextPollRequest returns appropriate next poll request based on poller configuration. +// Simple rules: +// 1. if sticky execution is disabled, always poll for regular task queue +// 2. otherwise: +// 2.1) if sticky task queue has backlog, always prefer to process sticky task first +// 2.2) poll from the task queue that has less pending requests (prefer sticky when they are the same). +// +// TODO: make this more smart to auto adjust based on poll latency +func (wtp *workflowTaskPoller) getNextPollRequest() (request *workflowservice.PollWorkflowTaskQueueRequest) { + taskQueue := &taskqueuepb.TaskQueue{ + Name: wtp.taskQueueName, + Kind: enumspb.TASK_QUEUE_KIND_NORMAL, + } + if wtp.stickyCacheSize > 0 { + wtp.requestLock.Lock() + if wtp.stickyBacklog > 0 || wtp.pendingStickyPollCount <= wtp.pendingRegularPollCount { + wtp.pendingStickyPollCount++ + taskQueue.Name = getWorkerTaskQueue(wtp.stickyUUID) + taskQueue.Kind = enumspb.TASK_QUEUE_KIND_STICKY + taskQueue.NormalName = wtp.taskQueueName + } else { + wtp.pendingRegularPollCount++ + } + wtp.requestLock.Unlock() + } + + builtRequest := &workflowservice.PollWorkflowTaskQueueRequest{ + Namespace: wtp.namespace, + TaskQueue: taskQueue, + Identity: wtp.identity, + BinaryChecksum: wtp.workerBuildID, + WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ + BuildId: wtp.workerBuildID, + UseVersioning: wtp.useBuildIDVersioning, + }, + } + if wtp.getCapabilities().BuildIdBasedVersioning { + builtRequest.BinaryChecksum = "" + } + return builtRequest +} + +// Poll the workflow task queue and update the num_poller metric +func (wtp *workflowTaskPoller) pollWorkflowTaskQueue(ctx context.Context, request *workflowservice.PollWorkflowTaskQueueRequest) (*workflowservice.PollWorkflowTaskQueueResponse, error) { + if request.TaskQueue.GetKind() == enumspb.TASK_QUEUE_KIND_NORMAL { + wtp.numNormalPollerMetric.increment() + defer wtp.numNormalPollerMetric.decrement() + } else { + wtp.numStickyPollerMetric.increment() + defer wtp.numStickyPollerMetric.decrement() + } + + return wtp.service.PollWorkflowTaskQueue(ctx, request) +} + +// Poll for a single workflow task from the service +func (wtp *workflowTaskPoller) poll(ctx context.Context) (taskForWorker, error) { + traceLog(func() { + wtp.logger.Debug("workflowTaskPoller::Poll") + }) + + request := wtp.getNextPollRequest() + defer wtp.release(request.TaskQueue.GetKind()) + + response, err := wtp.pollWorkflowTaskQueue(ctx, request) + if err != nil { + wtp.updateBacklog(request.TaskQueue.GetKind(), 0) + return nil, err + } + + if response == nil || len(response.TaskToken) == 0 { + // Emit using base scope as no workflow type information is available in the case of empty poll + wtp.metricsHandler.Counter(metrics.WorkflowTaskQueuePollEmptyCounter).Inc(1) + wtp.updateBacklog(request.TaskQueue.GetKind(), 0) + return &workflowTask{}, nil + } + + wtp.updateBacklog(request.TaskQueue.GetKind(), response.GetBacklogCountHint()) + + task := wtp.toWorkflowTask(response) + traceLog(func() { + var firstEventID int64 = -1 + if response.History != nil && len(response.History.Events) > 0 { + firstEventID = response.History.Events[0].GetEventId() + } + wtp.logger.Debug("workflowTaskPoller::Poll Succeed", + "StartedEventID", response.GetStartedEventId(), + "Attempt", response.GetAttempt(), + "FirstEventID", firstEventID, + "IsQueryTask", response.Query != nil) + }) + + metricsHandler := wtp.metricsHandler.WithTags(metrics.WorkflowTags(response.WorkflowType.GetName())) + metricsHandler.Counter(metrics.WorkflowTaskQueuePollSucceedCounter).Inc(1) + + scheduleToStartLatency := response.GetStartedTime().AsTime().Sub(response.GetScheduledTime().AsTime()) + metricsHandler.Timer(metrics.WorkflowTaskScheduleToStartLatency).Record(scheduleToStartLatency) + return task, nil +} + +func (wtp *workflowTaskPoller) toWorkflowTask(response *workflowservice.PollWorkflowTaskQueueResponse) *workflowTask { + historyIterator := &historyIteratorImpl{ + execution: response.WorkflowExecution, + nextPageToken: response.NextPageToken, + namespace: wtp.namespace, + service: wtp.service, + maxEventID: response.GetStartedEventId(), + metricsHandler: wtp.metricsHandler, + taskQueue: wtp.taskQueueName, + } + task := &workflowTask{ + task: response, + historyIterator: historyIterator, + } + return task +} + +func (h *historyIteratorImpl) GetNextPage() (*historypb.History, error) { + if h.iteratorFunc == nil { + h.iteratorFunc = newGetHistoryPageFunc( + context.Background(), + h.service, + h.namespace, + h.execution, + h.maxEventID, + h.metricsHandler, + h.taskQueue, + ) + } + + history, token, err := h.iteratorFunc(h.nextPageToken) + if err != nil { + return nil, err + } + h.nextPageToken = token + return history, nil +} + +func (h *historyIteratorImpl) Reset() { + h.nextPageToken = nil +} + +func (h *historyIteratorImpl) HasNextPage() bool { + return h.nextPageToken != nil +} + +func newGetHistoryPageFunc( + ctx context.Context, + service workflowservice.WorkflowServiceClient, + namespace string, + execution *commonpb.WorkflowExecution, + atWorkflowTaskCompletedEventID int64, + metricsHandler metrics.Handler, + taskQueue string, +) func(nextPageToken []byte) (*historypb.History, []byte, error) { + return func(nextPageToken []byte) (*historypb.History, []byte, error) { + var resp *workflowservice.GetWorkflowExecutionHistoryResponse + grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler( + metricsHandler.WithTags(metrics.RPCTags(metrics.NoneTagValue, metrics.NoneTagValue, taskQueue))), + defaultGrpcRetryParameters(ctx)) + defer cancel() + + resp, err := service.GetWorkflowExecutionHistory(grpcCtx, &workflowservice.GetWorkflowExecutionHistoryRequest{ + Namespace: namespace, + Execution: execution, + NextPageToken: nextPageToken, + }) + if err != nil { + return nil, nil, err + } + + var h *historypb.History + + if resp.RawHistory != nil { + h, err = serializer.DeserializeBlobDataToHistoryEvents(resp.RawHistory, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + if err != nil { + return nil, nil, nil + } + } else { + h = resp.History + } + + size := len(h.Events) + if size > 0 && atWorkflowTaskCompletedEventID > 0 && + h.Events[size-1].GetEventId() > atWorkflowTaskCompletedEventID { + first := h.Events[0].GetEventId() // eventIds start from 1 + h.Events = h.Events[:atWorkflowTaskCompletedEventID-first+1] + if h.Events[len(h.Events)-1].GetEventType() != enumspb.EVENT_TYPE_WORKFLOW_TASK_COMPLETED { + return nil, nil, fmt.Errorf("newGetHistoryPageFunc: atWorkflowTaskCompletedEventID(%v) "+ + "points to event that is not WorkflowTaskCompleted", atWorkflowTaskCompletedEventID) + } + return h, nil, nil + } + return h, resp.NextPageToken, nil + } +} + +func newActivityTaskPoller(taskHandler ActivityTaskHandler, service workflowservice.WorkflowServiceClient, params workerExecutionParameters) *activityTaskPoller { + return &activityTaskPoller{ + basePoller: basePoller{ + metricsHandler: params.MetricsHandler, + stopC: params.WorkerStopChannel, + workerBuildID: params.getBuildID(), + useBuildIDVersioning: params.UseBuildIDForVersioning, + capabilities: params.capabilities, + }, + taskHandler: taskHandler, + service: service, + namespace: params.Namespace, + taskQueueName: params.TaskQueue, + identity: params.Identity, + logger: params.Logger, + activitiesPerSecond: params.TaskQueueActivitiesPerSecond, + numPollerMetric: newNumPollerMetric(params.MetricsHandler, metrics.PollerTypeActivityTask), + } +} + +// Poll the activity task queue and update the num_poller metric +func (atp *activityTaskPoller) pollActivityTaskQueue(ctx context.Context, request *workflowservice.PollActivityTaskQueueRequest) (*workflowservice.PollActivityTaskQueueResponse, error) { + atp.numPollerMetric.increment() + defer atp.numPollerMetric.decrement() + + return atp.service.PollActivityTaskQueue(ctx, request) +} + +// Poll for a single activity task from the service +func (atp *activityTaskPoller) poll(ctx context.Context) (taskForWorker, error) { + traceLog(func() { + atp.logger.Debug("activityTaskPoller::Poll") + }) + request := &workflowservice.PollActivityTaskQueueRequest{ + Namespace: atp.namespace, + TaskQueue: &taskqueuepb.TaskQueue{Name: atp.taskQueueName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, + Identity: atp.identity, + TaskQueueMetadata: &taskqueuepb.TaskQueueMetadata{MaxTasksPerSecond: wrapperspb.Double(atp.activitiesPerSecond)}, + WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ + BuildId: atp.workerBuildID, + UseVersioning: atp.useBuildIDVersioning, + }, + } + + response, err := atp.pollActivityTaskQueue(ctx, request) + if err != nil { + return nil, err + } + if response == nil || len(response.TaskToken) == 0 { + // No activity info is available on empty poll. Emit using base scope. + atp.metricsHandler.Counter(metrics.ActivityPollNoTaskCounter).Inc(1) + return &activityTask{}, nil + } + + workflowType := response.WorkflowType.GetName() + activityType := response.ActivityType.GetName() + metricsHandler := atp.metricsHandler.WithTags(metrics.ActivityTags(workflowType, activityType, atp.taskQueueName)) + + scheduleToStartLatency := response.GetStartedTime().AsTime().Sub(response.GetCurrentAttemptScheduledTime().AsTime()) + metricsHandler.Timer(metrics.ActivityScheduleToStartLatency).Record(scheduleToStartLatency) + + return &activityTask{task: response}, nil +} + +// PollTask polls a new task +func (atp *activityTaskPoller) PollTask() (taskForWorker, error) { + // Get the task. + activityTask, err := atp.doPoll(atp.poll) + if err != nil { + return nil, err + } + return activityTask, nil +} + +// ProcessTask processes a new task +func (atp *activityTaskPoller) ProcessTask(task interface{}) error { + if atp.stopping() { + return errStop + } + + activityTask := task.(*activityTask) + if activityTask.task == nil { + // We didn't have task, poll might have timeout. + traceLog(func() { + atp.logger.Debug("Activity task unavailable") + }) + return nil + } + + workflowType := activityTask.task.WorkflowType.GetName() + activityType := activityTask.task.ActivityType.GetName() + activityMetricsHandler := atp.metricsHandler.WithTags(metrics.ActivityTags(workflowType, activityType, atp.taskQueueName)) + + executionStartTime := time.Now() + // Process the activity task. + request, err := atp.taskHandler.Execute(atp.taskQueueName, activityTask.task) + // err is returned in case of internal failure, such as unable to propagate context or context timeout. + if err != nil { + activityMetricsHandler.Counter(metrics.ActivityExecutionFailedCounter).Inc(1) + return err + } + // in case if activity execution failed, request should be of type RespondActivityTaskFailedRequest + if _, ok := request.(*workflowservice.RespondActivityTaskFailedRequest); ok { + activityMetricsHandler.Counter(metrics.ActivityExecutionFailedCounter).Inc(1) + } + activityMetricsHandler.Timer(metrics.ActivityExecutionLatency).Record(time.Since(executionStartTime)) + + if request == ErrActivityResultPending { + return nil + } + + rpcMetricsHandler := atp.metricsHandler.WithTags(metrics.RPCTags(workflowType, activityType, metrics.NoneTagValue)) + reportErr := reportActivityComplete(context.Background(), atp.service, request, rpcMetricsHandler) + if reportErr != nil { + traceLog(func() { + atp.logger.Debug("reportActivityComplete failed", tagError, reportErr) + }) + return reportErr + } + + if _, ok := request.(*workflowservice.RespondActivityTaskCompletedRequest); ok { + activityMetricsHandler. + Timer(metrics.ActivitySucceedEndToEndLatency). + Record(time.Since(activityTask.task.GetScheduledTime().AsTime())) + } + return nil +} + +func reportActivityComplete( + ctx context.Context, + service workflowservice.WorkflowServiceClient, + request interface{}, + rpcMetricsHandler metrics.Handler, +) error { + if request == nil { + // nothing to report + return nil + } + + var reportErr error + switch rqst := request.(type) { + case *workflowservice.RespondActivityTaskCanceledRequest: + grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler(rpcMetricsHandler), + defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := service.RespondActivityTaskCanceled(grpcCtx, rqst) + reportErr = err + case *workflowservice.RespondActivityTaskFailedRequest: + grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler(rpcMetricsHandler), defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := service.RespondActivityTaskFailed(grpcCtx, rqst) + reportErr = err + case *workflowservice.RespondActivityTaskCompletedRequest: + grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler(rpcMetricsHandler), + defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := service.RespondActivityTaskCompleted(grpcCtx, rqst) + reportErr = err + } + return reportErr +} + +func reportActivityCompleteByID( + ctx context.Context, + service workflowservice.WorkflowServiceClient, + request interface{}, + rpcMetricsHandler metrics.Handler, +) error { + if request == nil { + // nothing to report + return nil + } + + var reportErr error + switch request := request.(type) { + case *workflowservice.RespondActivityTaskCanceledByIdRequest: + grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler(rpcMetricsHandler), + defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := service.RespondActivityTaskCanceledById(grpcCtx, request) + reportErr = err + case *workflowservice.RespondActivityTaskFailedByIdRequest: + grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler(rpcMetricsHandler), + defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := service.RespondActivityTaskFailedById(grpcCtx, request) + reportErr = err + case *workflowservice.RespondActivityTaskCompletedByIdRequest: + grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler(rpcMetricsHandler), + defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := service.RespondActivityTaskCompletedById(grpcCtx, request) + reportErr = err + } + return reportErr +} + +func convertActivityResultToRespondRequest( + identity string, + taskToken []byte, + result *commonpb.Payloads, + err error, + dataConverter converter.DataConverter, + failureConverter converter.FailureConverter, + namespace string, + cancelAllowed bool, + versionStamp *commonpb.WorkerVersionStamp, +) interface{} { + if err == ErrActivityResultPending { + // activity result is pending and will be completed asynchronously. + // nothing to report at this point + return ErrActivityResultPending + } + + if err == nil { + return &workflowservice.RespondActivityTaskCompletedRequest{ + TaskToken: taskToken, + Result: result, + Identity: identity, + Namespace: namespace, + WorkerVersion: versionStamp, + } + } + + // Only respond with canceled if allowed + if cancelAllowed { + var canceledErr *CanceledError + if errors.As(err, &canceledErr) { + return &workflowservice.RespondActivityTaskCanceledRequest{ + TaskToken: taskToken, + Details: convertErrDetailsToPayloads(canceledErr.details, dataConverter), + Identity: identity, + Namespace: namespace, + WorkerVersion: versionStamp, + } + } + if errors.Is(err, context.Canceled) { + return &workflowservice.RespondActivityTaskCanceledRequest{ + TaskToken: taskToken, + Identity: identity, + Namespace: namespace, + WorkerVersion: versionStamp, + } + } + } + + // If a canceled error is returned but it wasn't allowed, we have to wrap in + // an unexpected-cancel application error + if _, isCanceledErr := err.(*CanceledError); isCanceledErr { + err = fmt.Errorf("unexpected activity cancel error: %w", err) + } + + return &workflowservice.RespondActivityTaskFailedRequest{ + TaskToken: taskToken, + Failure: failureConverter.ErrorToFailure(err), + Identity: identity, + Namespace: namespace, + WorkerVersion: versionStamp, + } +} + +func convertActivityResultToRespondRequestByID( + identity string, + namespace string, + workflowID string, + runID string, + activityID string, + result *commonpb.Payloads, + err error, + dataConverter converter.DataConverter, + failureConverter converter.FailureConverter, + cancelAllowed bool, +) interface{} { + if err == ErrActivityResultPending { + // activity result is pending and will be completed asynchronously. + // nothing to report at this point + return nil + } + + if err == nil { + return &workflowservice.RespondActivityTaskCompletedByIdRequest{ + Namespace: namespace, + WorkflowId: workflowID, + RunId: runID, + ActivityId: activityID, + Result: result, + Identity: identity, + } + } + + // Only respond with canceled if allowed + if cancelAllowed { + var canceledErr *CanceledError + if errors.As(err, &canceledErr) { + return &workflowservice.RespondActivityTaskCanceledByIdRequest{ + Namespace: namespace, + WorkflowId: workflowID, + RunId: runID, + ActivityId: activityID, + Details: convertErrDetailsToPayloads(canceledErr.details, dataConverter), + Identity: identity, + } + } + if errors.Is(err, context.Canceled) { + return &workflowservice.RespondActivityTaskCanceledByIdRequest{ + Namespace: namespace, + WorkflowId: workflowID, + RunId: runID, + ActivityId: activityID, + Identity: identity, + } + } + } + + // If a canceled error is returned but it wasn't allowed, we have to wrap in + // an unexpected-cancel application error + if _, isCanceledErr := err.(*CanceledError); isCanceledErr { + err = fmt.Errorf("unexpected activity cancel error: %w", err) + } + + return &workflowservice.RespondActivityTaskFailedByIdRequest{ + Namespace: namespace, + WorkflowId: workflowID, + RunId: runID, + ActivityId: activityID, + Failure: failureConverter.ErrorToFailure(err), + Identity: identity, + } +} + +func (wft *workflowTask) isEmpty() bool { + return wft.task == nil +} + +func (at *activityTask) isEmpty() bool { + return at.task == nil +} + +func (*localActivityTask) isEmpty() bool { + return false +} + +func (*eagerWorkflowTask) isEmpty() bool { + return false +} + +func (nt *nexusTask) isEmpty() bool { + return nt.task == nil +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_time.go b/vendor/go.temporal.io/sdk/internal/internal_time.go new file mode 100644 index 00000000000..6ea11f8f94b --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_time.go @@ -0,0 +1,65 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "time" +) + +// All code in this file is private to the package. + +type ( + // TimerID contains id of the timer + TimerID struct { + id string + } + + // WorkflowTimerClient wraps the async workflow timer functionality. + WorkflowTimerClient interface { + + // Now - Current time when the workflow task is started or replayed. + // the workflow need to use this for wall clock to make the flow logic deterministic. + Now() time.Time + + // NewTimer - Creates a new timer that will fire callback after d(resolution is in seconds). + // The callback indicates the error(TimerCanceledError) if the timer is canceled. + NewTimer(d time.Duration, options TimerOptions, callback ResultHandler) *TimerID + + // RequestCancelTimer - Requests cancel of a timer, this one doesn't wait for cancellation request + // to complete, instead invokes the ResultHandler with TimerCanceledError + // If the timer is not started then it is a no-operation. + RequestCancelTimer(timerID TimerID) + } +) + +func (i TimerID) String() string { + return i.id +} + +// ParseTimerID returns TimerID constructed from its string representation. +// The string representation should be obtained through TimerID.String() +func ParseTimerID(id string) (TimerID, error) { + return TimerID{id: id}, nil +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_update.go b/vendor/go.temporal.io/sdk/internal/internal_update.go new file mode 100644 index 00000000000..19f9d2ff3c4 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_update.go @@ -0,0 +1,495 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "errors" + "fmt" + "reflect" + + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + historypb "go.temporal.io/api/history/v1" + protocolpb "go.temporal.io/api/protocol/v1" + updatepb "go.temporal.io/api/update/v1" + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/protocol" +) + +type updateState string + +// WorkflowUpdateStage indicates the stage of an update request. +type WorkflowUpdateStage int + +const ( + // WorkflowUpdateStageUnspecified indicates the wait stage was not specified + WorkflowUpdateStageUnspecified WorkflowUpdateStage = iota + // WorkflowUpdateStageAdmitted indicates the update is admitted + WorkflowUpdateStageAdmitted + // WorkflowUpdateStageAccepted indicates the update is accepted + WorkflowUpdateStageAccepted + // WorkflowUpdateStageCompleted indicates the update is completed + WorkflowUpdateStageCompleted +) + +const ( + updateStateNew updateState = "New" + updateStateRequestInitiated updateState = "RequestScheduled" + updateStateAccepted updateState = "Accepted" + updateStateCompleted updateState = "Completed" + + updateProtocolV1 = "temporal.api.update.v1" +) + +type ( + // UpdateCallbacks supplies callbacks for the different stages of processing + // a workflow update. + UpdateCallbacks interface { + // Accept is called for an update after it has passed validation an + // before execution has started. + Accept() + + // Reject is called for an update if validation fails. + Reject(err error) + + // Complete is called for an update with the result of executing the + // update function. If the provided error is non-nil then the overall + // outcome is understood to be a failure. + Complete(success interface{}, err error) + } + + // UpdateScheduler allows an update state machine to spawn coroutines and + // yield itself as necessary. + UpdateScheduler interface { + // Spawn starts a new named coroutine, executing the given function f. + Spawn(ctx Context, name string, highPriority bool, f func(ctx Context)) Context + + // Yield returns control to the scheduler. + Yield(ctx Context, status string) + } + + // updateEnv encapsulates the utility functions needed by update protocol + // instance in order to implement the UpdateCallbacks interface. This + // interface is conveniently implemented by + // *workflowExecutionEventHandlerImpl. + updateEnv interface { + GetFailureConverter() converter.FailureConverter + GetDataConverter() converter.DataConverter + Send(*protocolpb.Message, ...msgSendOpt) + } + + // updateProtocol wraps an updateEnv and some protocol metadata to + // implement the UpdateCallbacks abstraction. It handles callbacks by + // sending protocol messages. + updateProtocol struct { + protoInstanceID string + clientIdentity string + initialRequest *updatepb.Request + requestMsgID string + requestSeqID int64 + scheduleUpdate func(name string, id string, args *commonpb.Payloads, header *commonpb.Header, callbacks UpdateCallbacks) + env updateEnv + state updateState + } + + // updateHandler is the underlying type that is registered into a workflow + // environment when the user-code in a workflow registers an update callback + // for a given name. It offers the ability to invoke the associated + // execution and validation functions. + updateHandler struct { + fn interface{} + validateFn interface{} + name string + unfinishedPolicy HandlerUnfinishedPolicy + description string + } +) + +// newUpdateResponder constructs an updateProtocolResponder instance to handle +// update callbacks. +func newUpdateProtocol( + protoInstanceID string, + scheduleUpdate func(name string, id string, args *commonpb.Payloads, header *commonpb.Header, callbacks UpdateCallbacks), + env updateEnv, +) *updateProtocol { + return &updateProtocol{ + protoInstanceID: protoInstanceID, + env: env, + scheduleUpdate: scheduleUpdate, + state: updateStateNew, + } +} + +func (up *updateProtocol) requireState(action string, valid ...updateState) { + for _, validState := range valid { + if up.state == validState { + return + } + } + panicIllegalState(fmt.Sprintf("[TMPRL1100] invalid action %q in update protocol %+v", action, up)) +} + +func (up *updateProtocol) HandleMessage(msg *protocolpb.Message) error { + var request updatepb.Request + if err := msg.Body.UnmarshalTo(&request); err != nil { + return err + } + up.initialRequest = &request + up.requireState("update request", updateStateNew) + up.requestMsgID = msg.GetId() + up.requestSeqID = msg.GetEventId() + input := up.initialRequest.GetInput() + up.scheduleUpdate(input.GetName(), up.initialRequest.GetMeta().GetUpdateId(), input.GetArgs(), input.GetHeader(), up) + up.state = updateStateRequestInitiated + return nil +} + +// Accept is called for an update after it has passed validation and +// before execution has started. +func (up *updateProtocol) Accept() { + up.requireState("accept", updateStateRequestInitiated) + up.env.Send(&protocolpb.Message{ + Id: up.protoInstanceID + "/accept", + ProtocolInstanceId: up.protoInstanceID, + Body: protocol.MustMarshalAny(&updatepb.Acceptance{ + AcceptedRequestMessageId: up.requestMsgID, + AcceptedRequestSequencingEventId: up.requestSeqID, + AcceptedRequest: up.initialRequest, + }), + }, withExpectedEventPredicate(up.checkAcceptedEvent)) + // Stop holding a reference to the initial request to allow it to be GCed + up.initialRequest = nil + up.state = updateStateAccepted +} + +// Reject is called for an update if validation fails. +func (up *updateProtocol) Reject(err error) { + up.requireState("reject", updateStateNew, updateStateRequestInitiated) + up.env.Send(&protocolpb.Message{ + Id: up.protoInstanceID + "/reject", + ProtocolInstanceId: up.protoInstanceID, + Body: protocol.MustMarshalAny(&updatepb.Rejection{ + RejectedRequestMessageId: up.requestMsgID, + RejectedRequestSequencingEventId: up.requestSeqID, + RejectedRequest: up.initialRequest, + Failure: up.env.GetFailureConverter().ErrorToFailure(err), + }), + }) + up.state = updateStateCompleted +} + +// Complete is called for an update with the result of executing the +// update function. +func (up *updateProtocol) Complete(success interface{}, outcomeErr error) { + up.requireState("complete", updateStateAccepted) + outcome := &updatepb.Outcome{} + if outcomeErr != nil { + outcome.Value = &updatepb.Outcome_Failure{ + Failure: up.env.GetFailureConverter().ErrorToFailure(outcomeErr), + } + } else { + success, err := up.env.GetDataConverter().ToPayloads(success) + if err != nil { + panic(err) + } + outcome.Value = &updatepb.Outcome_Success{ + Success: success, + } + } + up.env.Send(&protocolpb.Message{ + Id: up.protoInstanceID + "/complete", + ProtocolInstanceId: up.protoInstanceID, + Body: protocol.MustMarshalAny(&updatepb.Response{ + Meta: &updatepb.Meta{ + UpdateId: up.protoInstanceID, + Identity: up.clientIdentity, + }, + Outcome: outcome, + }), + }, withExpectedEventPredicate(up.checkCompletedEvent)) + up.state = updateStateCompleted +} + +func (up *updateProtocol) checkCompletedEvent(e *historypb.HistoryEvent) bool { + attrs := e.GetWorkflowExecutionUpdateCompletedEventAttributes() + if attrs == nil { + return false + } + return attrs.Meta.GetUpdateId() == up.protoInstanceID +} + +func (up *updateProtocol) checkAcceptedEvent(e *historypb.HistoryEvent) bool { + attrs := e.GetWorkflowExecutionUpdateAcceptedEventAttributes() + if attrs == nil { + return false + } + return attrs.GetProtocolInstanceId() == up.protoInstanceID && + attrs.AcceptedRequestMessageId == up.requestMsgID && + attrs.AcceptedRequestSequencingEventId == up.requestSeqID +} + +// defaultHandler receives the initial invocation of an update during WFT +// processing. The implementation will verify that an updateHandler exists for +// the supplied name (rejecting the update otherwise) and use the provided spawn +// function to create a new coroutine that will execute in the workflow context. +// The spawned coroutine is what will actually invoke the user-supplied callback +// functions for validation and execution. Update progress is emitted via calls +// into the UpdateCallbacks parameter. +func defaultUpdateHandler( + rootCtx Context, + name string, + id string, + serializedArgs *commonpb.Payloads, + header *commonpb.Header, + callbacks UpdateCallbacks, + scheduler UpdateScheduler, +) { + env := getWorkflowEnvironment(rootCtx) + ctx, err := workflowContextWithHeaderPropagated(rootCtx, header, env.GetContextPropagators()) + if err != nil { + callbacks.Reject(err) + return + } + eo := getWorkflowEnvOptions(ctx) + priorityUpdateHandling := env.TryUse(SDKPriorityUpdateHandling) + + updateRunner := func(ctx Context) { + updateInfo := UpdateInfo{ + ID: id, + Name: name, + } + ctx = WithValue(ctx, updateInfoContextKey, &updateInfo) + + eo := getWorkflowEnvOptions(ctx) + if len(eo.updateHandlers) == 0 && !priorityUpdateHandling { + scheduler.Yield(ctx, "yielding for initial handler registration") + } + handler, ok := eo.updateHandlers[name] + if !ok { + keys := make([]string, 0, len(eo.updateHandlers)) + for k := range eo.updateHandlers { + keys = append(keys, k) + } + callbacks.Reject(fmt.Errorf("unknown update %v. KnownUpdates=%v", name, keys)) + return + } + + args, err := decodeArgsToRawValues( + env.GetDataConverter(), + reflect.TypeOf(handler.fn), + serializedArgs, + ) + if err != nil { + callbacks.Reject(fmt.Errorf("unable to decode the input for update %q: %w", name, err)) + return + } + input := UpdateInput{Name: name, Args: args} + eo.runningUpdatesHandles[id] = updateInfo + defer func() { + delete(eo.runningUpdatesHandles, id) + }() + + envInterceptor := getWorkflowEnvironmentInterceptor(ctx) + if !IsReplaying(ctx) { + // we don't execute update validation during replay so that + // validation routines can change across versions + err = func() error { + defer getState(ctx).dispatcher.setIsReadOnly(false) + getState(ctx).dispatcher.setIsReadOnly(true) + return envInterceptor.inboundInterceptor.ValidateUpdate(ctx, &input) + }() + if err != nil { + callbacks.Reject(err) + return + } + } + callbacks.Accept() + success, err := envInterceptor.inboundInterceptor.ExecuteUpdate(ctx, &input) + callbacks.Complete(success, err) + } + + // If we suspect that handler registration has not occurred (e.g. + // because this update is part of the first workflow task and is being + // delivered before the workflow function itself has run and had a + // chance to register update handlers) then we queue updates + // to allow handler registration to occur. When a handler is registered the + // updates will be scheduled and ran. + if len(eo.updateHandlers) == 0 && priorityUpdateHandling { + env.QueueUpdate(name, func() { + scheduler.Spawn(ctx, name, priorityUpdateHandling, updateRunner) + }) + } else { + scheduler.Spawn(ctx, name, priorityUpdateHandling, updateRunner) + } + +} + +// newUpdateHandler instantiates a new updateHandler if the supplied handler and +// opts.Validator functions pass validation of their respective interfaces and +// that the two interfaces are themselves equivalent (allowing for them to +// differ by the presence/absence of a leading Context parameter). +func newUpdateHandler( + updateName string, + handler interface{}, + opts UpdateHandlerOptions, +) (*updateHandler, error) { + if err := validateUpdateHandlerFn(handler); err != nil { + return nil, err + } + var validateFn interface{} = func(...interface{}) error { return nil } + if opts.Validator != nil { + if err := validateValidatorFn(opts.Validator); err != nil { + return nil, err + } + if err := validateEquivalentParams(handler, opts.Validator); err != nil { + return nil, err + } + validateFn = opts.Validator + } + return &updateHandler{ + fn: handler, + validateFn: validateFn, + name: updateName, + unfinishedPolicy: opts.UnfinishedPolicy, + description: opts.Description, + }, nil +} + +// validate invokes the update's validation function. +func (h *updateHandler) validate(ctx Context, input []interface{}) (err error) { + defer func() { + if p := recover(); p != nil { + if p == panicIllegalAccessCoroutineState { + // Don't handle the panic since this error means the workflow state is + // likely corrupted and should be discarded. + panic(p) + } + st := getStackTraceRaw("update validator [panic]:", 7, 0) + err = newPanicError(fmt.Sprintf("update validator panic: %v", p), st) + } + }() + _, err = executeFunctionWithWorkflowContext(ctx, h.validateFn, input) + return err +} + +// execute executes the update itself. +func (h *updateHandler) execute(ctx Context, input []interface{}) (result interface{}, err error) { + return executeFunctionWithWorkflowContext(ctx, h.fn, input) +} + +// HasCompleted allows the completion status of the update protocol to be +// observed externally. +func (up *updateProtocol) HasCompleted() bool { + return up.state == updateStateCompleted +} + +// validateValidatorFn validates that the supplied interface +// +// 1. is a function +// 2. has exactly one return parameter +// 3. the one return parameter is of type `error` +func validateValidatorFn(fn interface{}) error { + fnType := reflect.TypeOf(fn) + if fnType.Kind() != reflect.Func { + return fmt.Errorf("validator must be function but was %s", fnType.Kind()) + } + + if fnType.NumOut() != 1 { + return fmt.Errorf( + "validator must return exactly 1 value (an error), but found %d return values", + fnType.NumOut(), + ) + } + + if !isError(fnType.Out(0)) { + return fmt.Errorf( + "return value of validator must be error but found %v", + fnType.Out(fnType.NumOut()-1).Kind(), + ) + } + return nil +} + +// validateUpdateHandlerFn validates that the supplied interface +// +// 1. is a function +// 2. has at least one parameter, the first of which is of type `workflow.Context` +// 3. has one or two return parameters, the last of which is of type `error` +// 4. if there are two return parameters, the first is a serializable type +func validateUpdateHandlerFn(fn interface{}) error { + fnType := reflect.TypeOf(fn) + if fnType.Kind() != reflect.Func { + return fmt.Errorf("handler must be function but was %s", fnType.Kind()) + } + if fnType.NumIn() == 0 { + return errors.New("first parameter of handler must be a workflow.Context") + } else if !isWorkflowContext(fnType.In(0)) { + return fmt.Errorf( + "first parameter of handler must be a workflow.Context but found %v", + fnType.In(0).Kind(), + ) + } + switch fnType.NumOut() { + case 1: + if !isError(fnType.Out(0)) { + return fmt.Errorf( + "last return value of handler must be error but found %v", + fnType.Out(0).Kind(), + ) + } + case 2: + if !isValidResultType(fnType.Out(0)) { + return fmt.Errorf( + "first return value of handler must be serializable but found: %v", + fnType.Out(0).Kind(), + ) + } + if !isError(fnType.Out(1)) { + return fmt.Errorf( + "last return value of handler must be error but found %v", + fnType.Out(1).Kind(), + ) + } + default: + return errors.New("update handler return signature must be a single " + + "error or a serializable result and error (i.e. (ResultType, error))") + } + return nil +} + +func updateLifeCycleStageToProto(l WorkflowUpdateStage) enumspb.UpdateWorkflowExecutionLifecycleStage { + switch l { + case WorkflowUpdateStageUnspecified: + return enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED + case WorkflowUpdateStageAdmitted: + return enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ADMITTED + case WorkflowUpdateStageAccepted: + return enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ACCEPTED + case WorkflowUpdateStageCompleted: + return enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED + default: + panic("unknown update lifecycle stage") + } +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_utils.go b/vendor/go.temporal.io/sdk/internal/internal_utils.go new file mode 100644 index 00000000000..15bff4857a1 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_utils.go @@ -0,0 +1,212 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +// All code in this file is private to the package. + +import ( + "context" + "fmt" + "os" + "os/signal" + "sync" + "syscall" + "time" + + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/internal/common/retry" + "google.golang.org/grpc/metadata" +) + +const ( + clientNameHeaderName = "client-name" + clientNameHeaderValue = "temporal-go" + clientVersionHeaderName = "client-version" + supportedServerVersionsHeaderName = "supported-server-versions" + + // defaultRPCTimeout is the default gRPC call timeout. + defaultRPCTimeout = 10 * time.Second + // minRPCTimeout is minimum gRPC call timeout allowed. + minRPCTimeout = 1 * time.Second + // maxRPCTimeout is maximum gRPC call timeout allowed (should not be less than defaultRPCTimeout). + maxRPCTimeout = 10 * time.Second +) + +// grpcContextBuilder stores all gRPC-specific parameters that will +// be stored inside of a context. +type grpcContextBuilder struct { + Timeout time.Duration + + // ParentContext to build the new context from. If empty, context.Background() is used. + // The new (child) context inherits a number of properties from the parent context: + // - context fields, accessible via `ctx.Value(key)` + ParentContext context.Context + + MetricsHandler metrics.Handler + + Headers metadata.MD + + IsLongPoll bool +} + +func (cb *grpcContextBuilder) Build() (context.Context, context.CancelFunc) { + ctx := cb.ParentContext + if ctx == nil { + ctx = context.Background() + } + if cb.Headers != nil { + ctx = metadata.NewOutgoingContext(ctx, cb.Headers) + } + if cb.MetricsHandler != nil { + ctx = context.WithValue(ctx, metrics.HandlerContextKey{}, cb.MetricsHandler) + } + ctx = context.WithValue(ctx, metrics.LongPollContextKey{}, cb.IsLongPoll) + var cancel context.CancelFunc + if cb.Timeout != time.Duration(0) { + ctx, cancel = context.WithTimeout(ctx, cb.Timeout) + } + + return ctx, cancel +} + +func grpcTimeout(timeout time.Duration) func(builder *grpcContextBuilder) { + return func(b *grpcContextBuilder) { + b.Timeout = timeout + } +} + +func grpcMetricsHandler(metricsHandler metrics.Handler) func(builder *grpcContextBuilder) { + return func(b *grpcContextBuilder) { + b.MetricsHandler = metricsHandler + } +} + +func grpcLongPoll(isLongPoll bool) func(builder *grpcContextBuilder) { + return func(b *grpcContextBuilder) { + b.IsLongPoll = isLongPoll + } +} + +func grpcContextValue(key interface{}, val interface{}) func(builder *grpcContextBuilder) { + return func(b *grpcContextBuilder) { + b.ParentContext = context.WithValue(b.ParentContext, key, val) + } +} + +func defaultGrpcRetryParameters(ctx context.Context) func(builder *grpcContextBuilder) { + return grpcContextValue(retry.ConfigKey, createDynamicServiceRetryPolicy(ctx).GrpcRetryConfig()) +} + +// newGRPCContext - Get context for gRPC calls. +func newGRPCContext(ctx context.Context, options ...func(builder *grpcContextBuilder)) (context.Context, context.CancelFunc) { + rpcTimeout := defaultRPCTimeout + + // Set rpc timeout less than context timeout to allow for retries when call gets lost + now := time.Now() + if deadline, ok := ctx.Deadline(); ok && deadline.After(now) { + rpcTimeout = deadline.Sub(now) / 2 + // Make sure to not set rpc timeout lower than minRPCTimeout + if rpcTimeout < minRPCTimeout { + rpcTimeout = minRPCTimeout + } else if rpcTimeout > maxRPCTimeout { + rpcTimeout = maxRPCTimeout + } + } + + builder := &grpcContextBuilder{ + ParentContext: ctx, + Timeout: rpcTimeout, + Headers: metadata.New(map[string]string{ + clientNameHeaderName: clientNameHeaderValue, + clientVersionHeaderName: SDKVersion, + supportedServerVersionsHeaderName: SupportedServerVersions, + }), + } + + for _, opt := range options { + opt(builder) + } + + return builder.Build() +} + +// GetWorkerIdentity gets a default identity for the worker. +func getWorkerIdentity(taskqueueName string) string { + return fmt.Sprintf("%d@%s@%s", os.Getpid(), getHostName(), taskqueueName) +} + +func getHostName() string { + hostName, err := os.Hostname() + if err != nil { + hostName = "Unknown" + } + return hostName +} + +func getWorkerTaskQueue(stickyUUID string) string { + // includes hostname for debuggability, stickyUUID guarantees the uniqueness + return fmt.Sprintf("%s:%s", getHostName(), stickyUUID) +} + +// AwaitWaitGroup calls Wait on the given wait +// Returns true if the Wait() call succeeded before the timeout +// Returns false if the Wait() did not return before the timeout +func awaitWaitGroup(wg *sync.WaitGroup, timeout time.Duration) bool { + doneC := make(chan struct{}) + + go func() { + wg.Wait() + close(doneC) + }() + + timer := time.NewTimer(timeout) + defer func() { timer.Stop() }() + + select { + case <-doneC: + return true + case <-timer.C: + return false + } +} + +// InterruptCh returns channel which will get data when system receives interrupt signal. Pass it to worker.Run() func to stop worker with Ctrl+C. +func InterruptCh() <-chan interface{} { + c := make(chan os.Signal, 1) + signal.Notify(c, os.Interrupt, syscall.SIGTERM) + + ret := make(chan interface{}, 1) + go func() { + s := <-c + ret <- s + close(ret) + }() + + return ret +} + +func getStringID(intID int64) string { + return fmt.Sprintf("%d", intID) +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_versioning_client.go b/vendor/go.temporal.io/sdk/internal/internal_versioning_client.go new file mode 100644 index 00000000000..4a70a560709 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_versioning_client.go @@ -0,0 +1,394 @@ +// The MIT License +// +// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "errors" + "time" + + "go.temporal.io/api/common/v1" + taskqueuepb "go.temporal.io/api/taskqueue/v1" + "go.temporal.io/api/workflowservice/v1" + + enumspb "go.temporal.io/api/enums/v1" +) + +// TaskQueueType specifies which category of tasks are associated with a queue. +type TaskQueueType int + +const ( + // TaskQueueTypeUnspecified indicates the task queue type was not specified. + TaskQueueTypeUnspecified = iota + // TaskQueueTypeWorkflow indicates the task queue is used for dispatching workflow tasks. + TaskQueueTypeWorkflow + // TaskQueueTypeActivity indicates the task queue is used for delivering activity tasks. + TaskQueueTypeActivity + // TaskQueueTypeNexus indicates the task queue is used for dispatching Nexus requests. + TaskQueueTypeNexus +) + +// BuildIDTaskReachability specifies which category of tasks may reach a versioned worker of a certain Build ID. +// +// Note: future activities who inherit their workflow's Build ID but not its task queue will not be +// accounted for reachability as server cannot know if they'll happen as they do not use +// assignment rules of their task queue. Same goes for Child Workflows or Continue-As-New Workflows +// who inherit the parent/previous workflow's Build ID but not its task queue. In those cases, make +// sure to query reachability for the parent/previous workflow's task queue as well. +type BuildIDTaskReachability int + +const ( + // BuildIDTaskReachabilityUnspecified indicates that task reachability was not reported. + BuildIDTaskReachabilityUnspecified = iota + // BuildIDTaskReachabilityReachable indicates that this Build ID may be used by new workflows or activities + // (based on versioning rules), or there are open workflows or backlogged activities assigned to it. + BuildIDTaskReachabilityReachable + // BuildIDTaskReachabilityClosedWorkflowsOnly specifies that this Build ID does not have open workflows + // and is not reachable by new workflows, but MAY have closed workflows within the namespace retention period. + // Not applicable to activity-only task queues. + BuildIDTaskReachabilityClosedWorkflowsOnly + // BuildIDTaskReachabilityUnreachable indicates that this Build ID is not used for new executions, nor + // it has been used by any existing execution within the retention period. + BuildIDTaskReachabilityUnreachable +) + +type ( + // TaskQueueVersionSelection is a task queue filter based on versioning. + // It is an optional component of [DescribeTaskQueueEnhancedOptions]. + TaskQueueVersionSelection struct { + // Include specific Build IDs. + BuildIDs []string + // Include the unversioned queue. + Unversioned bool + // Include all active versions. A version is considered active if, in the last few minutes, + // it has had new tasks or polls, or it has been the subject of certain task queue API calls. + AllActive bool + } + + // DescribeTaskQueueEnhancedOptions is the input to [Client.DescribeTaskQueueEnhanced]. + DescribeTaskQueueEnhancedOptions struct { + // Name of the task queue. Sticky queues are not supported. + TaskQueue string + // An optional queue selector based on versioning. If not provided, + // the result for the default Build ID will be returned. The default + // Build ID is the one mentioned in the first unconditional Assignment Rule. + // If there is no default Build ID, the result for the + // unversioned queue will be returned. + Versions *TaskQueueVersionSelection + // Task queue types to report info about. If not specified, all types are considered. + TaskQueueTypes []TaskQueueType + // Include list of pollers for requested task queue types and versions. + ReportPollers bool + // Include task reachability for the requested versions and all task types + // (task reachability is not reported per task type). + ReportTaskReachability bool + // Include task queue stats for requested task queue types and versions. + ReportStats bool + } + + // WorkerVersionCapabilities includes a worker's build identifier + // and whether it is choosing to use the versioning feature. + // It is an optional component of [TaskQueuePollerInfo]. + WorkerVersionCapabilities struct { + // Build ID of the worker. + BuildID string + // Whether the worker is using the versioning feature. + UseVersioning bool + } + + // TaskQueuePollerInfo provides information about a worker/client polling a task queue. + // It is used by [TaskQueueTypeInfo]. + TaskQueuePollerInfo struct { + // Time of the last poll. A value of zero means it was not set. + LastAccessTime time.Time + // The identity of the worker/client who is polling this task queue. + Identity string + // Polling rate. A value of zero means it was not set. + RatePerSecond float64 + // Optional poller versioning capabilities. Available when a worker has opted into the worker versioning feature. + WorkerVersionCapabilities *WorkerVersionCapabilities + } + + // TaskQueueStats contains statistics about task queue backlog and activity. + // + // For workflow task queue type, this result is partial because tasks sent to sticky queues are not included. Read + // comments above each metric to understand the impact of sticky queue exclusion on that metric accuracy. + TaskQueueStats struct { + // The approximate number of tasks backlogged in this task queue. May count expired tasks but eventually + // converges to the right value. Can be relied upon for scaling decisions. + // + // Special note for workflow task queue type: this metric does not count sticky queue tasks. However, because + // those tasks only remain valid for a few seconds, the inaccuracy becomes less significant as the backlog size + // grows. + ApproximateBacklogCount int64 + // Approximate age of the oldest task in the backlog based on the creation time of the task at the head of + // the queue. Can be relied upon for scaling decisions. + // + // Special note for workflow task queue type: this metric does not count sticky queue tasks. However, because + // those tasks only remain valid for a few seconds, they should not affect the result when backlog is older than + // few seconds. + ApproximateBacklogAge time.Duration + // Approximate *net* tasks per second added to the backlog, averaging the last 30 seconds. This is calculated as + // `TasksAddRate - TasksDispatchRate`. + // A positive value of `X` means the backlog is growing by about `X` tasks per second. A negative `-X` value means the + // backlog is shrinking by about `X` tasks per second. + // + // Special note for workflow task queue type: this metric does not count sticky queue tasks. However, because + // those tasks only remain valid for a few seconds, the inaccuracy becomes less significant as the backlog size + // or age grow. + BacklogIncreaseRate float32 + // Approximate tasks per second added to the task queue, averaging the last 30 seconds. This includes both + // backlogged and sync-matched tasks, but excludes the Eagerly dispatched workflow and activity tasks (see + // documentation for `client.StartWorkflowOptions.EnableEagerStart` and `worker.Options.DisableEagerActivities`.) + // + // The difference between `TasksAddRate` and `TasksDispatchRate` is a reliable metric for the rate at which + // backlog grows/shrinks. See `BacklogIncreaseRate`. + // + // Special note for workflow task queue type: this metric does not count sticky queue tasks. Hence, the reported + // value may be significantly lower than the actual number of workflow tasks added. Note that typically, only + // the first workflow task of each workflow goes to a normal queue, and the rest workflow tasks go to the sticky + // queue associated with a specific worker instance. Activity tasks always go to normal queues. + TasksAddRate float32 + // Approximate tasks per second dispatched to workers, averaging the last 30 seconds. This includes both + // backlogged and sync-matched tasks, but excludes the Eagerly dispatched workflow and activity tasks (see + // documentation for `client.StartWorkflowOptions.EnableEagerStart` and `worker.Options.DisableEagerActivities`.) + // + // The difference between `TasksAddRate` and `TasksDispatchRate` is a reliable metric for the rate at which + // backlog grows/shrinks. See `BacklogIncreaseRate`. + // + // Special note for workflow task queue type: this metric does not count sticky queue tasks. Hence, the reported + // value may be significantly lower than the actual number of workflow tasks dispatched. Note that typically, only + // the first workflow task of each workflow goes to a normal queue, and the rest workflow tasks go to the sticky + // queue associated with a specific worker instance. Activity tasks always go to normal queues. + TasksDispatchRate float32 + } + + // TaskQueueTypeInfo specifies task queue information per task type and Build ID. + // It is included in [TaskQueueVersionInfo]. + TaskQueueTypeInfo struct { + // Poller details for this task queue category. + Pollers []TaskQueuePollerInfo + Stats *TaskQueueStats + } + + // TaskQueueVersionInfo includes task queue information per Build ID. + // It is part of [TaskQueueDescription]. + TaskQueueVersionInfo struct { + // Task queue info per task type. + TypesInfo map[TaskQueueType]TaskQueueTypeInfo + // The category of tasks that may reach a versioned worker of a certain Build ID. + TaskReachability BuildIDTaskReachability + } + + // TaskQueueDescription is the response to [Client.DescribeTaskQueueEnhanced]. + TaskQueueDescription struct { + // Task queue information for each Build ID. Empty string as key value means unversioned. + VersionsInfo map[string]TaskQueueVersionInfo + } +) + +func (o *DescribeTaskQueueEnhancedOptions) validateAndConvertToProto(namespace string) (*workflowservice.DescribeTaskQueueRequest, error) { + if namespace == "" { + return nil, errors.New("missing namespace argument") + } + + if o.TaskQueue == "" { + return nil, errors.New("missing task queue field") + } + + taskQueueTypes := make([]enumspb.TaskQueueType, len(o.TaskQueueTypes)) + for i, t := range o.TaskQueueTypes { + taskQueueTypes[i] = taskQueueTypeToProto(t) + } + + opt := &workflowservice.DescribeTaskQueueRequest{ + Namespace: namespace, + TaskQueue: &taskqueuepb.TaskQueue{ + // Sticky queues not supported + Name: o.TaskQueue, + }, + ApiMode: enumspb.DESCRIBE_TASK_QUEUE_MODE_ENHANCED, + Versions: taskQueueVersionSelectionToProto(o.Versions), + TaskQueueTypes: taskQueueTypes, + ReportPollers: o.ReportPollers, + ReportTaskReachability: o.ReportTaskReachability, + ReportStats: o.ReportStats, + } + + return opt, nil +} + +func workerVersionCapabilitiesFromResponse(response *common.WorkerVersionCapabilities) *WorkerVersionCapabilities { + if response == nil { + return nil + } + + return &WorkerVersionCapabilities{ + BuildID: response.GetBuildId(), + UseVersioning: response.GetUseVersioning(), + } +} + +func pollerInfoFromResponse(response *taskqueuepb.PollerInfo) TaskQueuePollerInfo { + if response == nil { + return TaskQueuePollerInfo{} + } + + lastAccessTime := time.Time{} + if response.GetLastAccessTime() != nil { + lastAccessTime = response.GetLastAccessTime().AsTime() + } + + return TaskQueuePollerInfo{ + LastAccessTime: lastAccessTime, + Identity: response.GetIdentity(), + RatePerSecond: response.GetRatePerSecond(), + WorkerVersionCapabilities: workerVersionCapabilitiesFromResponse(response.GetWorkerVersionCapabilities()), + } +} + +func taskQueueTypeInfoFromResponse(response *taskqueuepb.TaskQueueTypeInfo) TaskQueueTypeInfo { + if response == nil { + return TaskQueueTypeInfo{} + } + + pollers := make([]TaskQueuePollerInfo, len(response.GetPollers())) + for i, pInfo := range response.GetPollers() { + pollers[i] = pollerInfoFromResponse(pInfo) + } + + return TaskQueueTypeInfo{ + Pollers: pollers, + Stats: statsFromResponse(response.Stats), + } +} + +func statsFromResponse(stats *taskqueuepb.TaskQueueStats) *TaskQueueStats { + if stats == nil { + return nil + } + + return &TaskQueueStats{ + ApproximateBacklogCount: stats.GetApproximateBacklogCount(), + ApproximateBacklogAge: stats.GetApproximateBacklogAge().AsDuration(), + TasksAddRate: stats.TasksAddRate, + TasksDispatchRate: stats.TasksDispatchRate, + BacklogIncreaseRate: stats.TasksAddRate - stats.TasksDispatchRate, + } +} + +func taskQueueVersionInfoFromResponse(response *taskqueuepb.TaskQueueVersionInfo) TaskQueueVersionInfo { + if response == nil { + return TaskQueueVersionInfo{} + } + + typesInfo := make(map[TaskQueueType]TaskQueueTypeInfo, len(response.GetTypesInfo())) + for taskType, tInfo := range response.GetTypesInfo() { + typesInfo[taskQueueTypeFromProto(enumspb.TaskQueueType(taskType))] = taskQueueTypeInfoFromResponse(tInfo) + } + + return TaskQueueVersionInfo{ + TypesInfo: typesInfo, + TaskReachability: buildIDTaskReachabilityFromProto(response.GetTaskReachability()), + } +} + +func detectTaskQueueEnhancedNotSupported(response *workflowservice.DescribeTaskQueueResponse) error { + // A server before 1.24 returns a non-enhanced proto, which only fills `pollers` and `taskQueueStatus` fields + if len(response.GetVersionsInfo()) == 0 && + (len(response.GetPollers()) > 0 || response.GetTaskQueueStatus() != nil) { + return errors.New("server does not support `DescribeTaskQueueEnhanced`") + } + return nil +} + +func taskQueueDescriptionFromResponse(response *workflowservice.DescribeTaskQueueResponse) TaskQueueDescription { + if response == nil { + return TaskQueueDescription{} + } + + versionsInfo := make(map[string]TaskQueueVersionInfo, len(response.GetVersionsInfo())) + for buildID, vInfo := range response.GetVersionsInfo() { + versionsInfo[buildID] = taskQueueVersionInfoFromResponse(vInfo) + } + + return TaskQueueDescription{ + VersionsInfo: versionsInfo, + } +} + +func taskQueueVersionSelectionToProto(s *TaskQueueVersionSelection) *taskqueuepb.TaskQueueVersionSelection { + if s == nil { + return nil + } + + return &taskqueuepb.TaskQueueVersionSelection{ + BuildIds: s.BuildIDs, + Unversioned: s.Unversioned, + AllActive: s.AllActive, + } +} + +func taskQueueTypeToProto(t TaskQueueType) enumspb.TaskQueueType { + switch t { + case TaskQueueTypeUnspecified: + return enumspb.TASK_QUEUE_TYPE_UNSPECIFIED + case TaskQueueTypeWorkflow: + return enumspb.TASK_QUEUE_TYPE_WORKFLOW + case TaskQueueTypeActivity: + return enumspb.TASK_QUEUE_TYPE_ACTIVITY + case TaskQueueTypeNexus: + return enumspb.TASK_QUEUE_TYPE_NEXUS + default: + panic("unknown task queue type") + } +} + +func taskQueueTypeFromProto(t enumspb.TaskQueueType) TaskQueueType { + switch t { + case enumspb.TASK_QUEUE_TYPE_UNSPECIFIED: + return TaskQueueTypeUnspecified + case enumspb.TASK_QUEUE_TYPE_WORKFLOW: + return TaskQueueTypeWorkflow + case enumspb.TASK_QUEUE_TYPE_ACTIVITY: + return TaskQueueTypeActivity + case enumspb.TASK_QUEUE_TYPE_NEXUS: + return TaskQueueTypeNexus + default: + panic("unknown task queue type from proto") + } +} + +func buildIDTaskReachabilityFromProto(r enumspb.BuildIdTaskReachability) BuildIDTaskReachability { + switch r { + case enumspb.BUILD_ID_TASK_REACHABILITY_UNSPECIFIED: + return BuildIDTaskReachabilityUnspecified + case enumspb.BUILD_ID_TASK_REACHABILITY_REACHABLE: + return BuildIDTaskReachabilityReachable + case enumspb.BUILD_ID_TASK_REACHABILITY_CLOSED_WORKFLOWS_ONLY: + return BuildIDTaskReachabilityClosedWorkflowsOnly + case enumspb.BUILD_ID_TASK_REACHABILITY_UNREACHABLE: + return BuildIDTaskReachabilityUnreachable + default: + panic("unknown task queue reachability") + } +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_worker.go b/vendor/go.temporal.io/sdk/internal/internal_worker.go new file mode 100644 index 00000000000..624e8b52850 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_worker.go @@ -0,0 +1,2009 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +// All code in this file is private to the package. + +import ( + "context" + "crypto/md5" + "encoding/hex" + "errors" + "fmt" + "io" + "math" + "os" + "reflect" + "runtime" + "strconv" + "strings" + "sync" + "sync/atomic" + "time" + + "github.com/golang/mock/gomock" + "github.com/nexus-rpc/sdk-go/nexus" + "github.com/pborman/uuid" + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + historypb "go.temporal.io/api/history/v1" + "go.temporal.io/api/temporalproto" + "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/api/workflowservicemock/v1" + "google.golang.org/protobuf/proto" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/internal/common/serializer" + "go.temporal.io/sdk/internal/common/util" + ilog "go.temporal.io/sdk/internal/log" + "go.temporal.io/sdk/log" +) + +const ( + // Set to 2 pollers for now, can adjust later if needed. The typical RTT (round-trip time) is below 1ms within data + // center. And the poll API latency is about 5ms. With 2 poller, we could achieve around 300~400 RPS. + defaultConcurrentPollRoutineSize = 2 + + defaultMaxConcurrentActivityExecutionSize = 1000 // Large concurrent activity execution size (1k) + defaultWorkerActivitiesPerSecond = 100000 // Large activity executions/sec (unlimited) + + defaultMaxConcurrentLocalActivityExecutionSize = 1000 // Large concurrent activity execution size (1k) + defaultWorkerLocalActivitiesPerSecond = 100000 // Large activity executions/sec (unlimited) + + defaultTaskQueueActivitiesPerSecond = 100000.0 // Large activity executions/sec (unlimited) + + defaultMaxConcurrentTaskExecutionSize = 1000 // hardcoded max task execution size. + defaultWorkerTaskExecutionRate = 100000 // Large task execution rate (unlimited) + + defaultPollerRate = 1000 + + defaultMaxConcurrentSessionExecutionSize = 1000 // Large concurrent session execution size (1k) + + defaultDeadlockDetectionTimeout = time.Second // By default kill workflow tasks that are running more than 1 sec. + // Unlimited deadlock detection timeout is used when we want to allow workflow tasks to run indefinitely, such + // as during debugging. + unlimitedDeadlockDetectionTimeout = math.MaxInt64 + + testTagsContextKey = "temporal-testTags" +) + +type ( + // WorkflowWorker wraps the code for hosting workflow types. + // And worker is mapped 1:1 with task queue. If the user want's to poll multiple + // task queue names they might have to manage 'n' workers for 'n' task queues. + workflowWorker struct { + executionParameters workerExecutionParameters + workflowService workflowservice.WorkflowServiceClient + poller taskPoller // taskPoller to poll and process the tasks. + worker *baseWorker + localActivityWorker *baseWorker + identity string + stopC chan struct{} + } + + // ActivityWorker wraps the code for hosting activity types. + // TODO: Worker doing heartbeating automatically while activity task is running + activityWorker struct { + executionParameters workerExecutionParameters + workflowService workflowservice.WorkflowServiceClient + poller taskPoller + worker *baseWorker + identity string + stopC chan struct{} + } + + // sessionWorker wraps the code for hosting session creation, completion and + // activities within a session. The creationWorker polls from a global taskqueue, + // while the activityWorker polls from a resource specific taskqueue. + sessionWorker struct { + creationWorker *activityWorker + activityWorker *activityWorker + } + + // Worker overrides. + workerOverrides struct { + workflowTaskHandler WorkflowTaskHandler + activityTaskHandler ActivityTaskHandler + slotSupplier SlotSupplier + } + + // workerExecutionParameters defines worker configure/execution options. + workerExecutionParameters struct { + // Namespace name. + Namespace string + + // Task queue name to poll. + TaskQueue string + + // The tuner for the worker. + Tuner WorkerTuner + + // Defines rate limiting on number of activity tasks that can be executed per second per worker. + WorkerActivitiesPerSecond float64 + + // MaxConcurrentActivityTaskQueuePollers is the max number of pollers for activity task queue. + MaxConcurrentActivityTaskQueuePollers int + + // MaxConcurrentWorkflowTaskQueuePollers is the max number of pollers for workflow task queue. + MaxConcurrentWorkflowTaskQueuePollers int + + // Defines rate limiting on number of local activities that can be executed per second per worker. + WorkerLocalActivitiesPerSecond float64 + + // TaskQueueActivitiesPerSecond is the throttling limit for activity tasks controlled by the server. + TaskQueueActivitiesPerSecond float64 + + // MaxConcurrentNexusTaskQueuePollers is the max number of pollers for the nexus task queue. + MaxConcurrentNexusTaskQueuePollers int + + // User can provide an identity for the debuggability. If not provided the framework has + // a default option. + Identity string + + // The worker's build ID used for versioning, if one was set. + WorkerBuildID string + // If true the worker is opting in to build ID based versioning. + UseBuildIDForVersioning bool + + MetricsHandler metrics.Handler + + Logger log.Logger + + // Enable logging in replay mode + EnableLoggingInReplay bool + + // Context to store user provided key/value pairs + UserContext context.Context + + // Context cancel function to cancel user context + UserContextCancel context.CancelFunc + + StickyScheduleToStartTimeout time.Duration + + // WorkflowPanicPolicy is used for configuring how client's workflow task handler deals with workflow + // code panicking which includes non backwards compatible changes to the workflow code without appropriate + // versioning (see workflow.GetVersion). + // The default behavior is to block workflow execution until the problem is fixed. + WorkflowPanicPolicy WorkflowPanicPolicy + + DataConverter converter.DataConverter + + FailureConverter converter.FailureConverter + + // WorkerStopTimeout is the time delay before hard terminate worker + WorkerStopTimeout time.Duration + + // WorkerStopChannel is a read only channel listen on worker close. The worker will close the channel before exit. + WorkerStopChannel <-chan struct{} + + // WorkerFatalErrorCallback is a callback for fatal errors that should stop + // the worker. + WorkerFatalErrorCallback func(error) + + // SessionResourceID is a unique identifier of the resource the session will consume + SessionResourceID string + + ContextPropagators []ContextPropagator + + // DeadlockDetectionTimeout specifies workflow task timeout. + DeadlockDetectionTimeout time.Duration + + DefaultHeartbeatThrottleInterval time.Duration + + MaxHeartbeatThrottleInterval time.Duration + + // Pointer to the shared worker cache + cache *WorkerCache + + eagerActivityExecutor *eagerActivityExecutor + + capabilities *workflowservice.GetSystemInfoResponse_Capabilities + } + + // HistoryJSONOptions are options for HistoryFromJSON. + HistoryJSONOptions struct { + // LastEventID, if set, will only load history up to this ID (inclusive). + LastEventID int64 + } +) + +var debugMode = os.Getenv("TEMPORAL_DEBUG") != "" + +// newWorkflowWorker returns an instance of the workflow worker. +func newWorkflowWorker(service workflowservice.WorkflowServiceClient, params workerExecutionParameters, ppMgr pressurePointMgr, registry *registry) *workflowWorker { + return newWorkflowWorkerInternal(service, params, ppMgr, nil, registry) +} + +func ensureRequiredParams(params *workerExecutionParameters) { + if params.Identity == "" { + params.Identity = getWorkerIdentity(params.TaskQueue) + } + if params.Logger == nil { + // create default logger if user does not supply one (should happen in tests only). + params.Logger = ilog.NewDefaultLogger() + params.Logger.Info("No logger configured for temporal worker. Created default one.") + } + if params.MetricsHandler == nil { + params.MetricsHandler = metrics.NopHandler + params.Logger.Info("No metrics handler configured for temporal worker. Use NopHandler as default.") + } + if params.DataConverter == nil { + params.DataConverter = converter.GetDefaultDataConverter() + params.Logger.Info("No DataConverter configured for temporal worker. Use default one.") + } + if params.FailureConverter == nil { + params.FailureConverter = GetDefaultFailureConverter() + } + if params.Tuner == nil { + // Err cannot happen since these slot numbers are guaranteed valid + params.Tuner, _ = NewFixedSizeTuner( + FixedSizeTunerOptions{ + NumWorkflowSlots: defaultMaxConcurrentTaskExecutionSize, + NumActivitySlots: defaultMaxConcurrentActivityExecutionSize, + NumLocalActivitySlots: defaultMaxConcurrentLocalActivityExecutionSize, + NumNexusSlots: defaultMaxConcurrentTaskExecutionSize, + }) + } +} + +// getBuildID returns either the user-defined build ID if it was provided, or an autogenerated one +// using getBinaryChecksum +func (params *workerExecutionParameters) getBuildID() string { + if params.WorkerBuildID != "" { + return params.WorkerBuildID + } + return getBinaryChecksum() +} + +// verifyNamespaceExist does a DescribeNamespace operation on the specified namespace with backoff/retry +func verifyNamespaceExist( + client workflowservice.WorkflowServiceClient, + metricsHandler metrics.Handler, + namespace string, + logger log.Logger, +) error { + ctx := context.Background() + if namespace == "" { + return errors.New("namespace cannot be empty") + } + grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler(metricsHandler), defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := client.DescribeNamespace(grpcCtx, &workflowservice.DescribeNamespaceRequest{Namespace: namespace}) + return err +} + +func newWorkflowWorkerInternal(service workflowservice.WorkflowServiceClient, params workerExecutionParameters, ppMgr pressurePointMgr, overrides *workerOverrides, registry *registry) *workflowWorker { + workerStopChannel := make(chan struct{}) + params.WorkerStopChannel = getReadOnlyChannel(workerStopChannel) + // Get a workflow task handler. + ensureRequiredParams(¶ms) + var taskHandler WorkflowTaskHandler + if overrides != nil && overrides.workflowTaskHandler != nil { + taskHandler = overrides.workflowTaskHandler + } else { + taskHandler = newWorkflowTaskHandler(params, ppMgr, registry) + } + return newWorkflowTaskWorkerInternal(taskHandler, taskHandler, service, params, workerStopChannel, registry.interceptors) +} + +func newWorkflowTaskWorkerInternal( + taskHandler WorkflowTaskHandler, + contextManager WorkflowContextManager, + service workflowservice.WorkflowServiceClient, + params workerExecutionParameters, + stopC chan struct{}, + interceptors []WorkerInterceptor, +) *workflowWorker { + ensureRequiredParams(¶ms) + poller := newWorkflowTaskPoller(taskHandler, contextManager, service, params) + worker := newBaseWorker(baseWorkerOptions{ + pollerCount: params.MaxConcurrentWorkflowTaskQueuePollers, + pollerRate: defaultPollerRate, + slotSupplier: params.Tuner.GetWorkflowTaskSlotSupplier(), + maxTaskPerSecond: defaultWorkerTaskExecutionRate, + taskWorker: poller, + workerType: "WorkflowWorker", + identity: params.Identity, + buildId: params.getBuildID(), + logger: params.Logger, + stopTimeout: params.WorkerStopTimeout, + fatalErrCb: params.WorkerFatalErrorCallback, + metricsHandler: params.MetricsHandler, + slotReservationData: slotReservationData{ + taskQueue: params.TaskQueue, + }, + }, + ) + + // laTunnel is the glue that hookup 3 parts + laTunnel := newLocalActivityTunnel(params.WorkerStopChannel) + + // 1) workflow handler will send local activity task to laTunnel + if handlerImpl, ok := taskHandler.(*workflowTaskHandlerImpl); ok { + handlerImpl.laTunnel = laTunnel + } + + // 2) local activity task poller will poll from laTunnel, and result will be pushed to laTunnel + localActivityTaskPoller := newLocalActivityPoller(params, laTunnel, interceptors) + localActivityWorker := newBaseWorker(baseWorkerOptions{ + pollerCount: 1, // 1 poller (from local channel) is enough for local activity + slotSupplier: params.Tuner.GetLocalActivitySlotSupplier(), + maxTaskPerSecond: params.WorkerLocalActivitiesPerSecond, + taskWorker: localActivityTaskPoller, + workerType: "LocalActivityWorker", + identity: params.Identity, + buildId: params.getBuildID(), + logger: params.Logger, + stopTimeout: params.WorkerStopTimeout, + fatalErrCb: params.WorkerFatalErrorCallback, + metricsHandler: params.MetricsHandler, + slotReservationData: slotReservationData{ + taskQueue: params.TaskQueue, + }, + }, + ) + + // 3) the result pushed to laTunnel will be sent as task to workflow worker to process. + worker.taskQueueCh = laTunnel.resultCh + + return &workflowWorker{ + executionParameters: params, + workflowService: service, + poller: poller, + worker: worker, + localActivityWorker: localActivityWorker, + identity: params.Identity, + stopC: stopC, + } +} + +// Start the worker. +func (ww *workflowWorker) Start() error { + err := verifyNamespaceExist(ww.workflowService, ww.executionParameters.MetricsHandler, ww.executionParameters.Namespace, ww.worker.logger) + if err != nil { + return err + } + ww.localActivityWorker.Start() + ww.worker.Start() + return nil // TODO: propagate error +} + +// Stop the worker. +func (ww *workflowWorker) Stop() { + close(ww.stopC) + // TODO: remove the stop methods in favor of the workerStopChannel + ww.localActivityWorker.Stop() + ww.worker.Stop() +} + +func newSessionWorker(service workflowservice.WorkflowServiceClient, params workerExecutionParameters, overrides *workerOverrides, env *registry, maxConcurrentSessionExecutionSize int) *sessionWorker { + if params.Identity == "" { + params.Identity = getWorkerIdentity(params.TaskQueue) + } + // For now resourceID is hidden from user so we will always create a unique one for each worker. + if params.SessionResourceID == "" { + params.SessionResourceID = uuid.New() + } + sessionEnvironment := newSessionEnvironment(params.SessionResourceID, maxConcurrentSessionExecutionSize) + + creationTaskqueue := getCreationTaskqueue(params.TaskQueue) + params.UserContext = context.WithValue(params.UserContext, sessionEnvironmentContextKey, sessionEnvironment) + params.TaskQueue = sessionEnvironment.GetResourceSpecificTaskqueue() + activityWorker := newActivityWorker(service, params, overrides, env, nil) + + params.MaxConcurrentActivityTaskQueuePollers = 1 + params.TaskQueue = creationTaskqueue + if overrides == nil { + overrides = &workerOverrides{} + } + // Although we have session token bucket to limit session size across creation + // and recreation, we also limit it here for creation only + overrides.slotSupplier, _ = NewFixedSizeSlotSupplier(maxConcurrentSessionExecutionSize) + creationWorker := newActivityWorker(service, params, overrides, env, sessionEnvironment.GetTokenBucket()) + + return &sessionWorker{ + creationWorker: creationWorker, + activityWorker: activityWorker, + } +} + +func (sw *sessionWorker) Start() error { + err := sw.creationWorker.Start() + if err != nil { + return err + } + + err = sw.activityWorker.Start() + if err != nil { + sw.creationWorker.Stop() + return err + } + return nil +} + +func (sw *sessionWorker) Stop() { + sw.creationWorker.Stop() + sw.activityWorker.Stop() +} + +func newActivityWorker( + service workflowservice.WorkflowServiceClient, + params workerExecutionParameters, + overrides *workerOverrides, + env *registry, + sessionTokenBucket *sessionTokenBucket, +) *activityWorker { + workerStopChannel := make(chan struct{}, 1) + params.WorkerStopChannel = getReadOnlyChannel(workerStopChannel) + ensureRequiredParams(¶ms) + + // Get a activity task handler. + var taskHandler ActivityTaskHandler + if overrides != nil && overrides.activityTaskHandler != nil { + taskHandler = overrides.activityTaskHandler + } else { + taskHandler = newActivityTaskHandler(service, params, env) + } + + poller := newActivityTaskPoller(taskHandler, service, params) + var slotSupplier SlotSupplier + if overrides != nil && overrides.slotSupplier != nil { + slotSupplier = overrides.slotSupplier + } else { + slotSupplier = params.Tuner.GetActivityTaskSlotSupplier() + } + + base := newBaseWorker( + baseWorkerOptions{ + pollerCount: params.MaxConcurrentActivityTaskQueuePollers, + pollerRate: defaultPollerRate, + slotSupplier: slotSupplier, + maxTaskPerSecond: params.WorkerActivitiesPerSecond, + taskWorker: poller, + workerType: "ActivityWorker", + identity: params.Identity, + buildId: params.getBuildID(), + logger: params.Logger, + stopTimeout: params.WorkerStopTimeout, + fatalErrCb: params.WorkerFatalErrorCallback, + userContextCancel: params.UserContextCancel, + metricsHandler: params.MetricsHandler, + sessionTokenBucket: sessionTokenBucket, + slotReservationData: slotReservationData{ + taskQueue: params.TaskQueue, + }, + }, + ) + return &activityWorker{ + executionParameters: params, + workflowService: service, + worker: base, + poller: poller, + identity: params.Identity, + stopC: workerStopChannel, + } +} + +// Start the worker. +func (aw *activityWorker) Start() error { + err := verifyNamespaceExist(aw.workflowService, aw.executionParameters.MetricsHandler, aw.executionParameters.Namespace, aw.worker.logger) + if err != nil { + return err + } + aw.worker.Start() + return nil // TODO: propagate errors +} + +// Stop the worker. +func (aw *activityWorker) Stop() { + close(aw.stopC) + aw.worker.Stop() +} + +type registry struct { + sync.Mutex + nexusServices map[string]*nexus.Service + workflowFuncMap map[string]interface{} + workflowAliasMap map[string]string + activityFuncMap map[string]activity + activityAliasMap map[string]string + interceptors []WorkerInterceptor +} + +type registryOptions struct { + disableAliasing bool +} + +func (r *registry) RegisterWorkflow(af interface{}) { + r.RegisterWorkflowWithOptions(af, RegisterWorkflowOptions{}) +} + +func (r *registry) RegisterWorkflowWithOptions( + wf interface{}, + options RegisterWorkflowOptions, +) { + // Support direct registration of WorkflowDefinition + factory, ok := wf.(WorkflowDefinitionFactory) + if ok { + if len(options.Name) == 0 { + panic("WorkflowDefinitionFactory must be registered with a name") + } + r.workflowFuncMap[options.Name] = factory + return + } + // Validate that it is a function + fnType := reflect.TypeOf(wf) + if err := validateFnFormat(fnType, true); err != nil { + panic(err) + } + fnName, _ := getFunctionName(wf) + alias := options.Name + registerName := fnName + if len(alias) > 0 { + registerName = alias + } + + r.Lock() + defer r.Unlock() + + if !options.DisableAlreadyRegisteredCheck { + if _, ok := r.workflowFuncMap[registerName]; ok { + panic(fmt.Sprintf("workflow name \"%v\" is already registered", registerName)) + } + } + r.workflowFuncMap[registerName] = wf + if len(alias) > 0 && r.workflowAliasMap != nil { + r.workflowAliasMap[fnName] = alias + } +} + +func (r *registry) RegisterActivity(af interface{}) { + r.RegisterActivityWithOptions(af, RegisterActivityOptions{}) +} + +func (r *registry) RegisterActivityWithOptions( + af interface{}, + options RegisterActivityOptions, +) { + // Support direct registration of activity + a, ok := af.(activity) + if ok { + if options.Name == "" { + panic("registration of activity interface requires name") + } + r.addActivityWithLock(options.Name, a) + return + } + // Validate that it is a function + fnType := reflect.TypeOf(af) + if fnType.Kind() == reflect.Ptr && fnType.Elem().Kind() == reflect.Struct { + registerErr := r.registerActivityStructWithOptions(af, options) + if registerErr != nil { + panic(registerErr) + } + return + } + if err := validateFnFormat(fnType, false); err != nil { + panic(err) + } + fnName, _ := getFunctionName(af) + alias := options.Name + registerName := fnName + if len(alias) > 0 { + registerName = alias + } + + r.Lock() + defer r.Unlock() + + if !options.DisableAlreadyRegisteredCheck { + if _, ok := r.activityFuncMap[registerName]; ok { + panic(fmt.Sprintf("activity type \"%v\" is already registered", registerName)) + } + } + r.activityFuncMap[registerName] = &activityExecutor{name: registerName, fn: af} + if len(alias) > 0 && r.activityAliasMap != nil { + r.activityAliasMap[fnName] = alias + } +} + +func (r *registry) registerActivityStructWithOptions(aStruct interface{}, options RegisterActivityOptions) error { + r.Lock() + defer r.Unlock() + + structValue := reflect.ValueOf(aStruct) + structType := structValue.Type() + count := 0 + for i := 0; i < structValue.NumMethod(); i++ { + methodValue := structValue.Method(i) + method := structType.Method(i) + // skip private method + if method.PkgPath != "" { + continue + } + name := method.Name + if err := validateFnFormat(method.Type, false); err != nil { + if options.SkipInvalidStructFunctions { + continue + } + + return fmt.Errorf("method %s of %s: %w", name, structType.Name(), err) + } + registerName := options.Name + name + if !options.DisableAlreadyRegisteredCheck { + if _, ok := r.getActivityNoLock(registerName); ok { + return fmt.Errorf("activity type \"%v\" is already registered", registerName) + } + } + r.activityFuncMap[registerName] = &activityExecutor{name: registerName, fn: methodValue.Interface()} + count++ + } + if count == 0 { + return fmt.Errorf("no activities (public methods) found at %v structure", structType.Name()) + } + return nil +} + +func (r *registry) RegisterNexusService(service *nexus.Service) { + if service.Name == "" { + panic(fmt.Errorf("tried to register a service with no name")) + } + + r.Lock() + defer r.Unlock() + + if _, ok := r.nexusServices[service.Name]; ok { + panic(fmt.Sprintf("service name \"%v\" is already registered", service.Name)) + } + r.nexusServices[service.Name] = service +} + +func (r *registry) getWorkflowAlias(fnName string) (string, bool) { + r.Lock() + defer r.Unlock() + alias, ok := r.workflowAliasMap[fnName] + return alias, ok +} + +func (r *registry) getWorkflowFn(fnName string) (interface{}, bool) { + r.Lock() + defer r.Unlock() + fn, ok := r.workflowFuncMap[fnName] + return fn, ok +} + +func (r *registry) getRegisteredWorkflowTypes() []string { + r.Lock() + defer r.Unlock() + var result []string + for t := range r.workflowFuncMap { + result = append(result, t) + } + return result +} + +func (r *registry) getActivityAlias(fnName string) (string, bool) { + r.Lock() + defer r.Unlock() + alias, ok := r.activityAliasMap[fnName] + return alias, ok +} + +func (r *registry) addActivityWithLock(fnName string, a activity) { + r.Lock() + defer r.Unlock() + r.activityFuncMap[fnName] = a +} + +func (r *registry) GetActivity(fnName string) (activity, bool) { + r.Lock() + defer r.Unlock() + a, ok := r.activityFuncMap[fnName] + return a, ok +} + +func (r *registry) getActivityNoLock(fnName string) (activity, bool) { + a, ok := r.activityFuncMap[fnName] + return a, ok +} + +func (r *registry) getRegisteredActivities() []activity { + r.Lock() + defer r.Unlock() + activities := make([]activity, 0, len(r.activityFuncMap)) + for _, a := range r.activityFuncMap { + activities = append(activities, a) + } + return activities +} + +func (r *registry) getRegisteredActivityTypes() []string { + r.Lock() + defer r.Unlock() + var result []string + for name := range r.activityFuncMap { + result = append(result, name) + } + return result +} + +func (r *registry) getWorkflowDefinition(wt WorkflowType) (WorkflowDefinition, error) { + lookup := wt.Name + if alias, ok := r.getWorkflowAlias(lookup); ok { + lookup = alias + } + wf, ok := r.getWorkflowFn(lookup) + if !ok { + supported := strings.Join(r.getRegisteredWorkflowTypes(), ", ") + return nil, fmt.Errorf("unable to find workflow type: %v. Supported types: [%v]", lookup, supported) + } + wdf, ok := wf.(WorkflowDefinitionFactory) + if ok { + return wdf.NewWorkflowDefinition(), nil + } + executor := &workflowExecutor{workflowType: lookup, fn: wf, interceptors: r.interceptors} + return newSyncWorkflowDefinition(executor), nil +} + +// Validate function parameters. +func validateFnFormat(fnType reflect.Type, isWorkflow bool) error { + if fnType.Kind() != reflect.Func { + return fmt.Errorf("expected a func as input but was %s", fnType.Kind()) + } + if isWorkflow { + if fnType.NumIn() < 1 { + return fmt.Errorf( + "expected at least one argument of type workflow.Context in function, found %d input arguments", + fnType.NumIn(), + ) + } + if !isWorkflowContext(fnType.In(0)) { + return fmt.Errorf("expected first argument to be workflow.Context but found %s", fnType.In(0)) + } + } else { + // For activities, check that workflow context is not accidentally provided + // Activities registered with structs will have their receiver as the first argument so confirm it is not + // in the first two arguments + for i := 0; i < fnType.NumIn() && i < 2; i++ { + if isWorkflowContext(fnType.In(i)) { + return fmt.Errorf("unexpected use of workflow context for an activity") + } + } + } + + // Return values + // We expect either + // , error + // (or) just error + if fnType.NumOut() < 1 || fnType.NumOut() > 2 { + return fmt.Errorf( + "expected function to return result, error or just error, but found %d return values", fnType.NumOut(), + ) + } + if fnType.NumOut() > 1 && !isValidResultType(fnType.Out(0)) { + return fmt.Errorf( + "expected function first return value to return valid type but found: %v", fnType.Out(0).Kind(), + ) + } + if !isError(fnType.Out(fnType.NumOut() - 1)) { + return fmt.Errorf( + "expected function second return value to return error but found %v", fnType.Out(fnType.NumOut()-1).Kind(), + ) + } + return nil +} + +func newRegistry() *registry { return newRegistryWithOptions(registryOptions{}) } + +func newRegistryWithOptions(options registryOptions) *registry { + r := ®istry{ + workflowFuncMap: make(map[string]interface{}), + activityFuncMap: make(map[string]activity), + nexusServices: make(map[string]*nexus.Service), + } + if !options.disableAliasing { + r.workflowAliasMap = make(map[string]string) + r.activityAliasMap = make(map[string]string) + } + return r +} + +// Wrapper to execute workflow functions. +type workflowExecutor struct { + workflowType string + fn interface{} + interceptors []WorkerInterceptor +} + +func (we *workflowExecutor) Execute(ctx Context, input *commonpb.Payloads) (*commonpb.Payloads, error) { + dataConverter := WithWorkflowContext(ctx, getWorkflowEnvOptions(ctx).DataConverter) + fnType := reflect.TypeOf(we.fn) + + args, err := decodeArgsToRawValues(dataConverter, fnType, input) + if err != nil { + return nil, fmt.Errorf( + "unable to decode the workflow function input payload with error: %w, function name: %v", + err, we.workflowType) + } + + envInterceptor := getWorkflowEnvironmentInterceptor(ctx) + envInterceptor.fn = we.fn + + // Execute and serialize result + result, err := envInterceptor.inboundInterceptor.ExecuteWorkflow(ctx, &ExecuteWorkflowInput{Args: args}) + var serializedResult *commonpb.Payloads + if err == nil && result != nil { + serializedResult, err = encodeArg(dataConverter, result) + } + return serializedResult, err +} + +// Wrapper to execute activity functions. +type activityExecutor struct { + name string + fn interface{} + skipInterceptors bool +} + +func (ae *activityExecutor) ActivityType() ActivityType { + return ActivityType{Name: ae.name} +} + +func (ae *activityExecutor) GetFunction() interface{} { + return ae.fn +} + +func (ae *activityExecutor) Execute(ctx context.Context, input *commonpb.Payloads) (*commonpb.Payloads, error) { + fnType := reflect.TypeOf(ae.fn) + dataConverter := getDataConverterFromActivityCtx(ctx) + + args, err := decodeArgsToRawValues(dataConverter, fnType, input) + if err != nil { + return nil, fmt.Errorf( + "unable to decode the activity function input payload with error: %w for function name: %v", + err, ae.name) + } + + return ae.ExecuteWithActualArgs(ctx, args) +} + +func (ae *activityExecutor) ExecuteWithActualArgs(ctx context.Context, args []interface{}) (*commonpb.Payloads, error) { + dataConverter := getDataConverterFromActivityCtx(ctx) + + envInterceptor := getActivityEnvironmentInterceptor(ctx) + envInterceptor.fn = ae.fn + + // Execute and serialize result + interceptor := envInterceptor.inboundInterceptor + if ae.skipInterceptors { + interceptor = envInterceptor + } + result, resultErr := interceptor.ExecuteActivity(ctx, &ExecuteActivityInput{Args: args}) + var serializedResult *commonpb.Payloads + if result != nil { + // As a special case, if the result is already a payload, just use it + var ok bool + if serializedResult, ok = result.(*commonpb.Payloads); !ok { + var err error + if serializedResult, err = encodeArg(dataConverter, result); err != nil { + return nil, err + } + } + } + return serializedResult, resultErr +} + +func getDataConverterFromActivityCtx(ctx context.Context) converter.DataConverter { + var dataConverter converter.DataConverter + + env := getActivityEnvironmentFromCtx(ctx) + if env != nil && env.dataConverter != nil { + dataConverter = env.dataConverter + } else { + dataConverter = converter.GetDefaultDataConverter() + } + return WithContext(ctx, dataConverter) +} + +func getNamespaceFromActivityCtx(ctx context.Context) string { + env := getActivityEnvironmentFromCtx(ctx) + if env == nil { + return "" + } + return env.workflowNamespace +} + +func getActivityEnvironmentFromCtx(ctx context.Context) *activityEnvironment { + if ctx == nil || ctx.Value(activityEnvContextKey) == nil { + return nil + } + return ctx.Value(activityEnvContextKey).(*activityEnvironment) +} + +// AggregatedWorker combines management of both workflowWorker and activityWorker worker lifecycle. +type AggregatedWorker struct { + // Stored for creating a nexus worker on Start. + executionParams workerExecutionParameters + // Memoized start function. Ensures start runs once and returns the same error when called multiple times. + memoizedStart func() error + + client *WorkflowClient + workflowWorker *workflowWorker + activityWorker *activityWorker + sessionWorker *sessionWorker + nexusWorker *nexusWorker + logger log.Logger + registry *registry + // Stores a boolean indicating whether the worker has already been started. + started atomic.Bool + stopC chan struct{} + fatalErr error + fatalErrLock sync.Mutex + capabilities *workflowservice.GetSystemInfoResponse_Capabilities +} + +// RegisterWorkflow registers workflow implementation with the AggregatedWorker +func (aw *AggregatedWorker) RegisterWorkflow(w interface{}) { + if aw.workflowWorker == nil { + panic("workflow worker disabled, cannot register workflow") + } + aw.registry.RegisterWorkflow(w) +} + +// RegisterWorkflowWithOptions registers workflow implementation with the AggregatedWorker +func (aw *AggregatedWorker) RegisterWorkflowWithOptions(w interface{}, options RegisterWorkflowOptions) { + if aw.workflowWorker == nil { + panic("workflow worker disabled, cannot register workflow") + } + aw.registry.RegisterWorkflowWithOptions(w, options) +} + +// RegisterActivity registers activity implementation with the AggregatedWorker +func (aw *AggregatedWorker) RegisterActivity(a interface{}) { + aw.registry.RegisterActivity(a) +} + +// RegisterActivityWithOptions registers activity implementation with the AggregatedWorker +func (aw *AggregatedWorker) RegisterActivityWithOptions(a interface{}, options RegisterActivityOptions) { + aw.registry.RegisterActivityWithOptions(a, options) +} + +func (aw *AggregatedWorker) RegisterNexusService(service *nexus.Service) { + if aw.started.Load() { + panic(errors.New("cannot register Nexus services after worker start")) + } + aw.registry.RegisterNexusService(service) +} + +// Start the worker in a non-blocking fashion. +// The actual work is done in the memoized "start" function to ensure duplicate calls are returned a consistent error. +func (aw *AggregatedWorker) Start() error { + aw.assertNotStopped() + return aw.memoizedStart() +} + +// start the worker. This method is memoized using sync.OnceValue in memoizedStart. +func (aw *AggregatedWorker) start() error { + aw.started.Store(true) + + if err := initBinaryChecksum(); err != nil { + return fmt.Errorf("failed to get executable checksum: %v", err) + } else if err = aw.client.ensureInitialized(context.Background()); err != nil { + return err + } + // Populate the capabilities. This should be the only time it is written too. + capabilities, err := aw.client.loadCapabilities(context.Background(), defaultGetSystemInfoTimeout) + if err != nil { + return err + } + proto.Merge(aw.capabilities, capabilities) + + if !util.IsInterfaceNil(aw.workflowWorker) { + if err := aw.workflowWorker.Start(); err != nil { + return err + } + if aw.client.eagerDispatcher != nil { + aw.client.eagerDispatcher.registerWorker(aw.workflowWorker) + } + } + if !util.IsInterfaceNil(aw.activityWorker) { + if err := aw.activityWorker.Start(); err != nil { + // stop workflow worker. + if !util.IsInterfaceNil(aw.workflowWorker) { + if aw.workflowWorker.worker.isWorkerStarted { + aw.workflowWorker.Stop() + } + } + return err + } + } + + if !util.IsInterfaceNil(aw.sessionWorker) && len(aw.registry.getRegisteredActivities()) > 0 { + aw.logger.Info("Starting session worker") + if err := aw.sessionWorker.Start(); err != nil { + // stop workflow worker and activity worker. + if !util.IsInterfaceNil(aw.workflowWorker) { + if aw.workflowWorker.worker.isWorkerStarted { + aw.workflowWorker.Stop() + } + } + if !util.IsInterfaceNil(aw.activityWorker) { + if aw.activityWorker.worker.isWorkerStarted { + aw.activityWorker.Stop() + } + } + return err + } + } + nexusServices := aw.registry.nexusServices + if len(nexusServices) > 0 { + reg := nexus.NewServiceRegistry() + for _, service := range nexusServices { + if err := reg.Register(service); err != nil { + return fmt.Errorf("failed to create a nexus worker: %w", err) + } + } + handler, err := reg.NewHandler() + if err != nil { + return fmt.Errorf("failed to create a nexus worker: %w", err) + } + aw.nexusWorker, err = newNexusWorker(nexusWorkerOptions{ + executionParameters: aw.executionParams, + client: aw.client, + workflowService: aw.client.workflowService, + handler: handler, + }) + if err != nil { + return fmt.Errorf("failed to create a nexus worker: %w", err) + } + if err := aw.nexusWorker.Start(); err != nil { + return fmt.Errorf("failed to start a nexus worker: %w", err) + } + } + aw.logger.Info("Started Worker") + return nil +} + +func (aw *AggregatedWorker) assertNotStopped() { + stopped := true + select { + case <-aw.stopC: + default: + stopped = false + } + if stopped { + panic("attempted to start a worker that has been stopped before") + } +} + +var ( + binaryChecksum string + binaryChecksumLock sync.Mutex +) + +// SetBinaryChecksum sets the identifier of the binary(aka BinaryChecksum). +// The identifier is mainly used in recording reset points when respondWorkflowTaskCompleted. For each workflow, the very first +// workflow task completed by a binary will be associated as a auto-reset point for the binary. So that when a customer wants to +// mark the binary as bad, the workflow will be reset to that point -- which means workflow will forget all progress generated +// by the binary. +// On another hand, once the binary is marked as bad, the bad binary cannot poll workflow queue and make any progress any more. +func SetBinaryChecksum(checksum string) { + binaryChecksumLock.Lock() + defer binaryChecksumLock.Unlock() + + binaryChecksum = checksum +} + +func initBinaryChecksum() error { + binaryChecksumLock.Lock() + defer binaryChecksumLock.Unlock() + + return initBinaryChecksumLocked() +} + +// callers MUST hold binaryChecksumLock before calling +func initBinaryChecksumLocked() error { + if len(binaryChecksum) > 0 { + return nil + } + + exec, err := os.Executable() + if err != nil { + return err + } + + f, err := os.Open(exec) + if err != nil { + return err + } + defer func() { + _ = f.Close() // error is unimportant as it is read-only + }() + + h := md5.New() + if _, err := io.Copy(h, f); err != nil { + return err + } + + checksum := h.Sum(nil) + binaryChecksum = hex.EncodeToString(checksum[:]) + + return nil +} + +func getBinaryChecksum() string { + binaryChecksumLock.Lock() + defer binaryChecksumLock.Unlock() + + if len(binaryChecksum) == 0 { + err := initBinaryChecksumLocked() + if err != nil { + panic(err) + } + } + + return binaryChecksum +} + +// Run the worker in a blocking fashion. Stop the worker when interruptCh receives signal. +// Pass worker.InterruptCh() to stop the worker with SIGINT or SIGTERM. +// Pass nil to stop the worker with external Stop() call. +// Pass any other `<-chan interface{}` and Run will wait for signal from that channel. +// Returns error if the worker fails to start or there is a fatal error +// during execution. +func (aw *AggregatedWorker) Run(interruptCh <-chan interface{}) error { + if err := aw.Start(); err != nil { + return err + } + select { + case s := <-interruptCh: + aw.logger.Info("Worker has been stopped.", "Signal", s) + aw.Stop() + case <-aw.stopC: + aw.fatalErrLock.Lock() + defer aw.fatalErrLock.Unlock() + // This may be nil if this wasn't stopped due to fatal error + return aw.fatalErr + } + return nil +} + +// Stop the worker. +func (aw *AggregatedWorker) Stop() { + // Only attempt stop if we haven't attempted before + select { + case <-aw.stopC: + return + default: + close(aw.stopC) + } + + if !util.IsInterfaceNil(aw.workflowWorker) { + aw.workflowWorker.Stop() + } + if !util.IsInterfaceNil(aw.activityWorker) { + aw.activityWorker.Stop() + } + if !util.IsInterfaceNil(aw.sessionWorker) { + aw.sessionWorker.Stop() + } + + aw.logger.Info("Stopped Worker") +} + +// WorkflowReplayer is used to replay workflow code from an event history +type WorkflowReplayer struct { + registry *registry + dataConverter converter.DataConverter + failureConverter converter.FailureConverter + contextPropagators []ContextPropagator + enableLoggingInReplay bool + disableDeadlockDetection bool + mu sync.Mutex + workflowExecutionResults map[string]*commonpb.Payloads +} + +// WorkflowReplayerOptions are options for creating a workflow replayer. +type WorkflowReplayerOptions struct { + // Optional custom data converter to provide for replay. If not set, the + // default converter is used. + DataConverter converter.DataConverter + + FailureConverter converter.FailureConverter + + // Optional: Sets ContextPropagators that allows users to control the context information passed through a workflow + // default: nil + ContextPropagators []ContextPropagator + + // Interceptors to apply to the worker. Earlier interceptors wrap later + // interceptors. + Interceptors []WorkerInterceptor + + // Disable aliasing during registration. This should be set if it was set on + // worker.Options.DisableRegistrationAliasing when originally run. See + // documentation for that field for more information. + DisableRegistrationAliasing bool + + // Optional: Enable logging in replay. + // In the workflow code you can use workflow.GetLogger(ctx) to write logs. By default, the logger will skip log + // entry during replay mode so you won't see duplicate logs. This option will enable the logging in replay mode. + // This is only useful for debugging purpose. + // default: false + EnableLoggingInReplay bool + + // Optional: Disable the default 1 second deadlock detection timeout. This option can be used to step through + // workflow code with multiple breakpoints in a debugger. + DisableDeadlockDetection bool +} + +// ReplayWorkflowHistoryOptions are options for replaying a workflow. +type ReplayWorkflowHistoryOptions struct { + // OriginalExecution - Overide the workflow execution details used for replay. + // Optional + OriginalExecution WorkflowExecution +} + +// NewWorkflowReplayer creates an instance of the WorkflowReplayer. +func NewWorkflowReplayer(options WorkflowReplayerOptions) (*WorkflowReplayer, error) { + registry := newRegistryWithOptions(registryOptions{disableAliasing: options.DisableRegistrationAliasing}) + registry.interceptors = options.Interceptors + return &WorkflowReplayer{ + registry: registry, + dataConverter: options.DataConverter, + failureConverter: options.FailureConverter, + contextPropagators: options.ContextPropagators, + enableLoggingInReplay: options.EnableLoggingInReplay, + disableDeadlockDetection: options.DisableDeadlockDetection, + workflowExecutionResults: make(map[string]*commonpb.Payloads), + }, nil +} + +// RegisterWorkflow registers workflow function to replay +func (aw *WorkflowReplayer) RegisterWorkflow(w interface{}) { + aw.registry.RegisterWorkflow(w) +} + +// RegisterWorkflowWithOptions registers workflow function with custom workflow name to replay +func (aw *WorkflowReplayer) RegisterWorkflowWithOptions(w interface{}, options RegisterWorkflowOptions) { + aw.registry.RegisterWorkflowWithOptions(w, options) +} + +// ReplayWorkflowHistoryWithOptions executes a single workflow task for the given history. +// Use for testing the backwards compatibility of code changes and troubleshooting workflows in a debugger. +// The logger is an optional parameter. Defaults to the noop logger. +func (aw *WorkflowReplayer) ReplayWorkflowHistoryWithOptions(logger log.Logger, history *historypb.History, options ReplayWorkflowHistoryOptions) error { + if logger == nil { + logger = ilog.NewDefaultLogger() + } + + controller := gomock.NewController(ilog.NewTestReporter(logger)) + service := workflowservicemock.NewMockWorkflowServiceClient(controller) + + return aw.replayWorkflowHistory(logger, service, ReplayNamespace, options.OriginalExecution, history) +} + +// ReplayWorkflowHistory executes a single workflow task for the given history. +// Use for testing the backwards compatibility of code changes and troubleshooting workflows in a debugger. +// The logger is an optional parameter. Defaults to the noop logger. +func (aw *WorkflowReplayer) ReplayWorkflowHistory(logger log.Logger, history *historypb.History) error { + return aw.ReplayWorkflowHistoryWithOptions(logger, history, ReplayWorkflowHistoryOptions{}) +} + +// ReplayWorkflowHistoryFromJSONFile executes a single workflow task for the given json history file. +// Use for testing the backwards compatibility of code changes and troubleshooting workflows in a debugger. +// The logger is an optional parameter. Defaults to the noop logger. +func (aw *WorkflowReplayer) ReplayWorkflowHistoryFromJSONFile(logger log.Logger, jsonfileName string) error { + return aw.ReplayPartialWorkflowHistoryFromJSONFile(logger, jsonfileName, 0) +} + +// ReplayPartialWorkflowHistoryFromJSONFile executes a single workflow task for the given json history file upto provided +// lastEventID(inclusive). +// Use for testing the backwards compatibility of code changes and troubleshooting workflows in a debugger. +// The logger is an optional parameter. Defaults to the noop logger. +func (aw *WorkflowReplayer) ReplayPartialWorkflowHistoryFromJSONFile(logger log.Logger, jsonfileName string, lastEventID int64) error { + history, err := extractHistoryFromFile(jsonfileName, lastEventID) + if err != nil { + return err + } + + if logger == nil { + logger = ilog.NewDefaultLogger() + } + + controller := gomock.NewController(ilog.NewTestReporter(logger)) + service := workflowservicemock.NewMockWorkflowServiceClient(controller) + + return aw.replayWorkflowHistory(logger, service, ReplayNamespace, WorkflowExecution{}, history) +} + +// ReplayWorkflowExecution replays workflow execution loading it from Temporal service. +func (aw *WorkflowReplayer) ReplayWorkflowExecution(ctx context.Context, service workflowservice.WorkflowServiceClient, logger log.Logger, namespace string, execution WorkflowExecution) error { + if logger == nil { + logger = ilog.NewDefaultLogger() + } + + sharedExecution := &commonpb.WorkflowExecution{ + RunId: execution.RunID, + WorkflowId: execution.ID, + } + request := &workflowservice.GetWorkflowExecutionHistoryRequest{ + Namespace: namespace, + Execution: sharedExecution, + } + var history historypb.History + for { + resp, err := service.GetWorkflowExecutionHistory(ctx, request) + if err != nil { + return err + } + currHistory := resp.History + if resp.RawHistory != nil { + currHistory, err = serializer.DeserializeBlobDataToHistoryEvents(resp.RawHistory, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + if err != nil { + return err + } + } + if currHistory == nil { + break + } + history.Events = append(history.Events, currHistory.Events...) + if len(resp.NextPageToken) == 0 { + break + } + request.NextPageToken = resp.NextPageToken + } + return aw.replayWorkflowHistory(logger, service, namespace, execution, &history) +} + +// GetWorkflowResult get the result of a succesfully replayed workflow. +func (aw *WorkflowReplayer) GetWorkflowResult(workflowID string, valuePtr interface{}) error { + aw.mu.Lock() + defer aw.mu.Unlock() + if workflowID == "" { + workflowID = "ReplayId" + } + payloads, ok := aw.workflowExecutionResults[workflowID] + if !ok { + return errors.New("workflow result not found") + } + dc := aw.dataConverter + if dc == nil { + dc = converter.GetDefaultDataConverter() + } + return dc.FromPayloads(payloads, valuePtr) +} + +func (aw *WorkflowReplayer) replayWorkflowHistory(logger log.Logger, service workflowservice.WorkflowServiceClient, namespace string, originalExecution WorkflowExecution, history *historypb.History) error { + taskQueue := "ReplayTaskQueue" + events := history.Events + if events == nil { + return errors.New("empty events") + } + if len(events) < 3 { + return errors.New("at least 3 events expected in the history") + } + first := events[0] + if first.GetEventType() != enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED { + return errors.New("first event is not WorkflowExecutionStarted") + } + last := events[len(events)-1] + + attr := first.GetWorkflowExecutionStartedEventAttributes() + if attr == nil { + return errors.New("corrupted WorkflowExecutionStarted") + } + workflowType := attr.WorkflowType + execution := &commonpb.WorkflowExecution{ + RunId: uuid.NewRandom().String(), + WorkflowId: "ReplayId", + } + if originalExecution.ID != "" { + execution.WorkflowId = originalExecution.ID + } + if originalExecution.RunID != "" { + execution.RunId = originalExecution.RunID + } else if first.GetWorkflowExecutionStartedEventAttributes().GetOriginalExecutionRunId() != "" { + execution.RunId = first.GetWorkflowExecutionStartedEventAttributes().GetOriginalExecutionRunId() + } + + if first.GetWorkflowExecutionStartedEventAttributes().GetTaskQueue().GetName() != "" { + taskQueue = first.GetWorkflowExecutionStartedEventAttributes().GetTaskQueue().GetName() + } + + task := &workflowservice.PollWorkflowTaskQueueResponse{ + Attempt: 1, + TaskToken: []byte("ReplayTaskToken"), + WorkflowType: workflowType, + WorkflowExecution: execution, + History: history, + PreviousStartedEventId: math.MaxInt64, + } + + iterator := &historyIteratorImpl{ + nextPageToken: task.NextPageToken, + execution: task.WorkflowExecution, + namespace: ReplayNamespace, + service: service, + maxEventID: task.GetStartedEventId(), + taskQueue: taskQueue, + } + cache := NewWorkerCache() + params := workerExecutionParameters{ + Namespace: namespace, + TaskQueue: taskQueue, + Identity: "replayID", + Logger: logger, + cache: cache, + DataConverter: aw.dataConverter, + FailureConverter: aw.failureConverter, + ContextPropagators: aw.contextPropagators, + EnableLoggingInReplay: aw.enableLoggingInReplay, + // Hardcoding NopHandler avoids "No metrics handler configured for temporal worker" + // logs during replay. + MetricsHandler: metrics.NopHandler, + capabilities: &workflowservice.GetSystemInfoResponse_Capabilities{ + SignalAndQueryHeader: true, + InternalErrorDifferentiation: true, + ActivityFailureIncludeHeartbeat: true, + SupportsSchedules: true, + EncodedFailureAttributes: true, + UpsertMemo: true, + EagerWorkflowStart: true, + SdkMetadata: true, + }, + } + if aw.disableDeadlockDetection { + params.DeadlockDetectionTimeout = math.MaxInt64 + } + taskHandler := newWorkflowTaskHandler(params, nil, aw.registry) + wfctx, err := taskHandler.GetOrCreateWorkflowContext(task, iterator) + defer wfctx.Unlock(err) + if err != nil { + return err + } + resp, err := taskHandler.ProcessWorkflowTask(&workflowTask{task: task, historyIterator: iterator}, wfctx, nil) + if err != nil { + return err + } + + if failedReq, ok := resp.(*workflowservice.RespondWorkflowTaskFailedRequest); ok { + return fmt.Errorf("replay workflow failed with failure: %v", failedReq.GetFailure()) + } + + if last.GetEventType() != enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED && last.GetEventType() != enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW { + return nil + } + + if resp != nil { + completeReq, ok := resp.(*workflowservice.RespondWorkflowTaskCompletedRequest) + if ok { + for _, d := range completeReq.Commands { + if d.GetCommandType() == enumspb.COMMAND_TYPE_CONTINUE_AS_NEW_WORKFLOW_EXECUTION { + if last.GetEventType() == enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW { + return nil + } + } + if d.GetCommandType() == enumspb.COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION { + if last.GetEventType() == enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED { + aw.mu.Lock() + defer aw.mu.Unlock() + aw.workflowExecutionResults[execution.WorkflowId] = d.GetCompleteWorkflowExecutionCommandAttributes().Result + return nil + } + } + } + } + } + return fmt.Errorf("replay workflow doesn't return the same result as the last event, resp: %[1]T{%[1]v}, last: %[2]T{%[2]v}", resp, last) +} + +// HistoryFromJSON deserializes history from a reader of JSON bytes. This does +// not close the reader if it is closeable. +func HistoryFromJSON(r io.Reader, lastEventID int64) (*historypb.History, error) { + // We set DiscardUnknown here because the history may have been created by a previous + // version of our protos + opts := temporalproto.CustomJSONUnmarshalOptions{ + DiscardUnknown: true, + } + bs, err := io.ReadAll(r) + if err != nil { + return nil, err + } + + hist := &historypb.History{} + if err := opts.Unmarshal(bs, hist); err != nil { + return nil, err + } + + // If there is a last event ID, slice the rest off + if lastEventID > 0 { + for i, event := range hist.Events { + if event.EventId == lastEventID { + // Inclusive + hist.Events = hist.Events[:i+1] + break + } + } + } + return hist, nil +} + +func extractHistoryFromFile(jsonfileName string, lastEventID int64) (hist *historypb.History, err error) { + reader, err := os.Open(jsonfileName) + if err != nil { + return nil, err + } + defer func() { + closeErr := reader.Close() + if closeErr != nil && err == nil { + err = closeErr + } else if closeErr != nil { + ilog.NewDefaultLogger().Warn("failed to close json file", "path", jsonfileName, "error", closeErr) + } + }() + + opts := temporalproto.CustomJSONUnmarshalOptions{ + DiscardUnknown: true, + } + + bs, err := io.ReadAll(reader) + if err != nil { + return nil, err + } + + hist = &historypb.History{} + if err := opts.Unmarshal(bs, hist); err != nil { + return nil, err + } + + // If there is a last event ID, slice the rest off + if lastEventID > 0 { + for i, event := range hist.Events { + if event.EventId == lastEventID { + // Inclusive + hist.Events = hist.Events[:i+1] + break + } + } + } + + return hist, err +} + +// NewAggregatedWorker returns an instance to manage both activity and workflow workers +func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options WorkerOptions) *AggregatedWorker { + setClientDefaults(client) + setWorkerOptionsDefaults(&options) + ctx := options.BackgroundActivityContext + if ctx == nil { + ctx = context.Background() + } + backgroundActivityContext, backgroundActivityContextCancel := context.WithCancel(ctx) + + // If max-concurrent workflow pollers is 1, the worker will only do + // sticky-queue requests and never regular-queue requests. We disallow the + // value of 1 here. + if options.MaxConcurrentWorkflowTaskPollers == 1 { + panic("cannot set MaxConcurrentWorkflowTaskPollers to 1") + } + + // If max-concurrent workflow task execution size is 1, the worker will only do + // sticky-queue requests and never regular-queue requests. This is because we + // limit the number of running pollers to MaxConcurrentWorkflowTaskExecutionSize. + // We disallow the value of 1 here. + if options.MaxConcurrentWorkflowTaskExecutionSize == 1 { + panic("cannot set MaxConcurrentWorkflowTaskExecutionSize to 1") + } + + // Sessions are not currently compatible with worker versioning + // See: https://github.com/temporalio/sdk-go/issues/1227 + if options.EnableSessionWorker && options.UseBuildIDForVersioning { + panic("cannot set both EnableSessionWorker and UseBuildIDForVersioning") + } + + // Need reference to result for fatal error handler + var aw *AggregatedWorker + fatalErrorCallback := func(err error) { + // Set the fatal error if not already set + aw.fatalErrLock.Lock() + alreadySet := aw.fatalErr != nil + if !alreadySet { + aw.fatalErr = err + } + aw.fatalErrLock.Unlock() + // Only do the rest if not already set + if !alreadySet { + // Invoke the callback if present + if options.OnFatalError != nil { + options.OnFatalError(err) + } + // Stop the worker if not already stopped + select { + case <-aw.stopC: + default: + aw.Stop() + } + } + } + // Because of lazy clients we need to wait till the worker runs to fetch the capabilities. + // All worker systems that depend on the capabilities to process workflow/activity tasks + // should take a pointer to this struct and wait for it to be populated when the worker is run. + var capabilities workflowservice.GetSystemInfoResponse_Capabilities + + cache := NewWorkerCache() + workerParams := workerExecutionParameters{ + Namespace: client.namespace, + TaskQueue: taskQueue, + Tuner: options.Tuner, + WorkerActivitiesPerSecond: options.WorkerActivitiesPerSecond, + MaxConcurrentActivityTaskQueuePollers: options.MaxConcurrentActivityTaskPollers, + WorkerLocalActivitiesPerSecond: options.WorkerLocalActivitiesPerSecond, + MaxConcurrentWorkflowTaskQueuePollers: options.MaxConcurrentWorkflowTaskPollers, + MaxConcurrentNexusTaskQueuePollers: options.MaxConcurrentNexusTaskPollers, + Identity: client.identity, + WorkerBuildID: options.BuildID, + UseBuildIDForVersioning: options.UseBuildIDForVersioning, + MetricsHandler: client.metricsHandler.WithTags(metrics.TaskQueueTags(taskQueue)), + Logger: client.logger, + EnableLoggingInReplay: options.EnableLoggingInReplay, + UserContext: backgroundActivityContext, + UserContextCancel: backgroundActivityContextCancel, + StickyScheduleToStartTimeout: options.StickyScheduleToStartTimeout, + TaskQueueActivitiesPerSecond: options.TaskQueueActivitiesPerSecond, + WorkflowPanicPolicy: options.WorkflowPanicPolicy, + DataConverter: client.dataConverter, + FailureConverter: client.failureConverter, + WorkerStopTimeout: options.WorkerStopTimeout, + WorkerFatalErrorCallback: fatalErrorCallback, + ContextPropagators: client.contextPropagators, + DeadlockDetectionTimeout: options.DeadlockDetectionTimeout, + DefaultHeartbeatThrottleInterval: options.DefaultHeartbeatThrottleInterval, + MaxHeartbeatThrottleInterval: options.MaxHeartbeatThrottleInterval, + cache: cache, + eagerActivityExecutor: newEagerActivityExecutor(eagerActivityExecutorOptions{ + disabled: options.DisableEagerActivities, + taskQueue: taskQueue, + maxConcurrent: options.MaxConcurrentEagerActivityExecutionSize, + }), + capabilities: &capabilities, + } + + if options.Identity != "" { + workerParams.Identity = options.Identity + } + + ensureRequiredParams(&workerParams) + workerParams.Logger = log.With(workerParams.Logger, + tagNamespace, client.namespace, + tagTaskQueue, taskQueue, + tagWorkerID, workerParams.Identity, + ) + if workerParams.WorkerBuildID != "" { + // Add worker build ID to the logs if it's set by user + workerParams.Logger = log.With(workerParams.Logger, + tagBuildID, workerParams.WorkerBuildID, + ) + } + + processTestTags(&options, &workerParams) + + // worker specific registry + registry := newRegistryWithOptions(registryOptions{disableAliasing: options.DisableRegistrationAliasing}) + // Build set of interceptors using the applicable client ones first (being + // careful not to append to the existing slice) + registry.interceptors = make([]WorkerInterceptor, 0, len(client.workerInterceptors)+len(options.Interceptors)) + registry.interceptors = append(append(registry.interceptors, client.workerInterceptors...), options.Interceptors...) + + // workflow factory. + var workflowWorker *workflowWorker + if !options.DisableWorkflowWorker { + testTags := getTestTags(options.BackgroundActivityContext) + if len(testTags) > 0 { + workflowWorker = newWorkflowWorkerWithPressurePoints(client.workflowService, workerParams, testTags, registry) + } else { + workflowWorker = newWorkflowWorker(client.workflowService, workerParams, nil, registry) + } + } + + // activity types. + var activityWorker *activityWorker + if !options.LocalActivityWorkerOnly { + activityWorker = newActivityWorker(client.workflowService, workerParams, nil, registry, nil) + workerParams.eagerActivityExecutor.activityWorker = activityWorker.worker + } + + var sessionWorker *sessionWorker + if options.EnableSessionWorker && !options.LocalActivityWorkerOnly { + sessionWorker = newSessionWorker(client.workflowService, workerParams, nil, registry, options.MaxConcurrentSessionExecutionSize) + registry.RegisterActivityWithOptions(sessionCreationActivity, RegisterActivityOptions{ + Name: sessionCreationActivityName, + }) + registry.RegisterActivityWithOptions(sessionCompletionActivity, RegisterActivityOptions{ + Name: sessionCompletionActivityName, + }) + } + + aw = &AggregatedWorker{ + client: client, + workflowWorker: workflowWorker, + activityWorker: activityWorker, + sessionWorker: sessionWorker, + logger: workerParams.Logger, + registry: registry, + stopC: make(chan struct{}), + capabilities: &capabilities, + executionParams: workerParams, + } + aw.memoizedStart = sync.OnceValue(aw.start) + return aw +} + +func processTestTags(wOptions *WorkerOptions, ep *workerExecutionParameters) { + testTags := getTestTags(wOptions.BackgroundActivityContext) + if testTags != nil { + if paramsOverride, ok := testTags[workerOptionsConfig]; ok { + for key, val := range paramsOverride { + switch key { + case workerOptionsConfigConcurrentPollRoutineSize: + if size, err := strconv.Atoi(val); err == nil { + ep.MaxConcurrentActivityTaskQueuePollers = size + ep.MaxConcurrentWorkflowTaskQueuePollers = size + } + } + } + } + } +} + +func isWorkflowContext(inType reflect.Type) bool { + // NOTE: We don't expect any one to derive from workflow context. + return inType == reflect.TypeOf((*Context)(nil)).Elem() +} + +func isValidResultType(inType reflect.Type) bool { + // https://golang.org/pkg/reflect/#Kind + switch inType.Kind() { + case reflect.Func, reflect.Chan, reflect.UnsafePointer: + return false + } + + return true +} + +func isError(inType reflect.Type) bool { + errorElem := reflect.TypeOf((*error)(nil)).Elem() + return inType != nil && inType.Implements(errorElem) +} + +func getFunctionName(i interface{}) (name string, isMethod bool) { + if fullName, ok := i.(string); ok { + return fullName, false + } + fullName := runtime.FuncForPC(reflect.ValueOf(i).Pointer()).Name() + // Full function name that has a struct pointer receiver has the following format + // .(*). + isMethod = strings.ContainsAny(fullName, "*") + elements := strings.Split(fullName, ".") + shortName := elements[len(elements)-1] + // This allows to call activities by method pointer + // Compiler adds -fm suffix to a function name which has a receiver + // Note that this works even if struct pointer used to get the function is nil + // It is possible because nil receivers are allowed. + // For example: + // var a *Activities + // ExecuteActivity(ctx, a.Foo) + // will call this function which is going to return "Foo" + return strings.TrimSuffix(shortName, "-fm"), isMethod +} + +func getActivityFunctionName(r *registry, i interface{}) string { + result, _ := getFunctionName(i) + if alias, ok := r.getActivityAlias(result); ok { + result = alias + } + return result +} + +func getWorkflowFunctionName(r *registry, workflowFunc interface{}) (string, error) { + fnName := "" + fType := reflect.TypeOf(workflowFunc) + switch getKind(fType) { + case reflect.String: + fnName = reflect.ValueOf(workflowFunc).String() + case reflect.Func: + fnName, _ = getFunctionName(workflowFunc) + if alias, ok := r.getWorkflowAlias(fnName); ok { + fnName = alias + } + default: + return "", fmt.Errorf("invalid type 'workflowFunc' parameter provided, it can be either worker function or function name: %v", workflowFunc) + } + + return fnName, nil +} + +func getReadOnlyChannel(c chan struct{}) <-chan struct{} { + return c +} + +func setWorkerOptionsDefaults(options *WorkerOptions) { + if options.Tuner != nil { + if options.MaxConcurrentWorkflowTaskExecutionSize != 0 || + options.MaxConcurrentActivityExecutionSize != 0 || + options.MaxConcurrentLocalActivityExecutionSize != 0 || + options.MaxConcurrentNexusTaskExecutionSize != 0 { + panic("cannot set MaxConcurrentWorkflowTaskExecutionSize, MaxConcurrentActivityExecutionSize, MaxConcurrentLocalActivityExecutionSize, or MaxConcurrentNexusTaskExecutionSize with Tuner") + } + } + maxConcurrentWFT := options.MaxConcurrentWorkflowTaskExecutionSize + maxConcurrentAct := options.MaxConcurrentActivityExecutionSize + maxConcurrentLA := options.MaxConcurrentLocalActivityExecutionSize + maxConcurrentNexus := options.MaxConcurrentNexusTaskExecutionSize + if options.MaxConcurrentActivityExecutionSize <= 0 { + maxConcurrentAct = defaultMaxConcurrentActivityExecutionSize + } + if options.WorkerActivitiesPerSecond == 0 { + options.WorkerActivitiesPerSecond = defaultWorkerActivitiesPerSecond + } + if options.MaxConcurrentActivityTaskPollers <= 0 { + options.MaxConcurrentActivityTaskPollers = defaultConcurrentPollRoutineSize + } + if options.MaxConcurrentWorkflowTaskExecutionSize <= 0 { + maxConcurrentWFT = defaultMaxConcurrentTaskExecutionSize + } + if options.MaxConcurrentWorkflowTaskPollers <= 0 { + options.MaxConcurrentWorkflowTaskPollers = defaultConcurrentPollRoutineSize + } + if options.MaxConcurrentLocalActivityExecutionSize <= 0 { + maxConcurrentLA = defaultMaxConcurrentLocalActivityExecutionSize + } + if options.WorkerLocalActivitiesPerSecond == 0 { + options.WorkerLocalActivitiesPerSecond = defaultWorkerLocalActivitiesPerSecond + } + if options.TaskQueueActivitiesPerSecond == 0 { + options.TaskQueueActivitiesPerSecond = defaultTaskQueueActivitiesPerSecond + } else { + // Disable eager activities when the task queue rate limit is set because + // the server does not rate limit eager activities. + options.DisableEagerActivities = true + } + if options.MaxConcurrentNexusTaskPollers <= 0 { + options.MaxConcurrentNexusTaskPollers = defaultConcurrentPollRoutineSize + } + if options.MaxConcurrentNexusTaskExecutionSize <= 0 { + maxConcurrentNexus = defaultMaxConcurrentTaskExecutionSize + } + if options.StickyScheduleToStartTimeout.Seconds() == 0 { + options.StickyScheduleToStartTimeout = stickyWorkflowTaskScheduleToStartTimeoutSeconds * time.Second + } + if options.MaxConcurrentSessionExecutionSize == 0 { + options.MaxConcurrentSessionExecutionSize = defaultMaxConcurrentSessionExecutionSize + } + if options.DeadlockDetectionTimeout == 0 { + if debugMode { + options.DeadlockDetectionTimeout = unlimitedDeadlockDetectionTimeout + } else { + options.DeadlockDetectionTimeout = defaultDeadlockDetectionTimeout + } + } + if options.DefaultHeartbeatThrottleInterval == 0 { + options.DefaultHeartbeatThrottleInterval = defaultDefaultHeartbeatThrottleInterval + } + if options.MaxHeartbeatThrottleInterval == 0 { + options.MaxHeartbeatThrottleInterval = defaultMaxHeartbeatThrottleInterval + } + if options.Tuner == nil { + // Err cannot happen since these slot numbers are guaranteed valid + options.Tuner, _ = NewFixedSizeTuner(FixedSizeTunerOptions{ + NumWorkflowSlots: maxConcurrentWFT, + NumActivitySlots: maxConcurrentAct, + NumLocalActivitySlots: maxConcurrentLA, + NumNexusSlots: maxConcurrentNexus}) + + } +} + +// setClientDefaults should be needed only in unit tests. +func setClientDefaults(client *WorkflowClient) { + if client.dataConverter == nil { + client.dataConverter = converter.GetDefaultDataConverter() + } + if client.namespace == "" { + client.namespace = DefaultNamespace + } + if client.metricsHandler == nil { + client.metricsHandler = metrics.NopHandler + } +} + +// getTestTags returns the test tags in the context. +func getTestTags(ctx context.Context) map[string]map[string]string { + if ctx != nil { + env := ctx.Value(testTagsContextKey) + if env != nil { + return env.(map[string]map[string]string) + } + } + return nil +} + +// Same as executeFunction but injects the workflow context as the first +// parameter if the function takes it (regardless of existing parameters). +func executeFunctionWithWorkflowContext(ctx Context, fn interface{}, args []interface{}) (interface{}, error) { + if fnType := reflect.TypeOf(fn); fnType.NumIn() > 0 && isWorkflowContext(fnType.In(0)) { + args = append([]interface{}{ctx}, args...) + } + return executeFunction(fn, args) +} + +// Same as executeFunction but injects the context as the first parameter if the +// function takes it (regardless of existing parameters). +func executeFunctionWithContext(ctx context.Context, fn interface{}, args []interface{}) (interface{}, error) { + if fnType := reflect.TypeOf(fn); fnType.NumIn() > 0 && isActivityContext(fnType.In(0)) { + args = append([]interface{}{ctx}, args...) + } + return executeFunction(fn, args) +} + +// Executes function and ensures that there is always 1 or 2 results and second +// result is error. +func executeFunction(fn interface{}, args []interface{}) (interface{}, error) { + fnValue := reflect.ValueOf(fn) + reflectArgs := make([]reflect.Value, len(args)) + for i, arg := range args { + // If the argument is nil, use zero value + if arg == nil { + reflectArgs[i] = reflect.New(fnValue.Type().In(i)).Elem() + } else { + reflectArgs[i] = reflect.ValueOf(arg) + } + } + retValues := fnValue.Call(reflectArgs) + + // Expect either error or (result, error) + if len(retValues) == 0 || len(retValues) > 2 { + fnName, _ := getFunctionName(fn) + return nil, fmt.Errorf( + "the function: %v signature returns %d results, it is expecting to return either error or (result, error)", + fnName, len(retValues)) + } + // Convert error + var err error + if errResult := retValues[len(retValues)-1].Interface(); errResult != nil { + var ok bool + if err, ok = errResult.(error); !ok { + return nil, fmt.Errorf( + "failed to serialize error result as it is not of error interface: %v", + errResult) + } + } + // If there are two results, convert the first only if it's not a nil pointer + var res interface{} + if len(retValues) > 1 && (retValues[0].Kind() != reflect.Ptr || !retValues[0].IsNil()) { + res = retValues[0].Interface() + } + return res, err +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_worker_base.go b/vendor/go.temporal.io/sdk/internal/internal_worker_base.go new file mode 100644 index 00000000000..d83a06e39fb --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_worker_base.go @@ -0,0 +1,595 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +// All code in this file is private to the package. + +import ( + "context" + "errors" + "fmt" + "sync" + "time" + + commonpb "go.temporal.io/api/common/v1" + "go.temporal.io/api/serviceerror" + "golang.org/x/time/rate" + + "go.temporal.io/sdk/internal/common/retry" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/backoff" + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/log" +) + +const ( + retryPollOperationInitialInterval = 200 * time.Millisecond + retryPollOperationMaxInterval = 10 * time.Second + retryPollResourceExhaustedInitialInterval = time.Second + retryPollResourceExhaustedMaxInterval = 10 * time.Second + // How long the same poll task error can remain suppressed + lastPollTaskErrSuppressTime = 1 * time.Minute +) + +var ( + pollOperationRetryPolicy = createPollRetryPolicy() + pollResourceExhaustedRetryPolicy = createPollResourceExhaustedRetryPolicy() + retryLongPollGracePeriod = 2 * time.Minute +) + +var errStop = errors.New("worker stopping") + +type ( + // ResultHandler that returns result + ResultHandler func(result *commonpb.Payloads, err error) + // LocalActivityResultHandler that returns local activity result + LocalActivityResultHandler func(lar *LocalActivityResultWrapper) + + // LocalActivityResultWrapper contains result of a local activity + LocalActivityResultWrapper struct { + Err error + Result *commonpb.Payloads + Attempt int32 + Backoff time.Duration + } + + executeNexusOperationParams struct { + client NexusClient + operation string + input *commonpb.Payload + options NexusOperationOptions + nexusHeader map[string]string + } + + // WorkflowEnvironment Represents the environment for workflow. + // Should only be used within the scope of workflow definition. + WorkflowEnvironment interface { + AsyncActivityClient + LocalActivityClient + WorkflowTimerClient + SideEffect(f func() (*commonpb.Payloads, error), callback ResultHandler) + GetVersion(changeID string, minSupported, maxSupported Version) Version + WorkflowInfo() *WorkflowInfo + TypedSearchAttributes() SearchAttributes + Complete(result *commonpb.Payloads, err error) + RegisterCancelHandler(handler func()) + RequestCancelChildWorkflow(namespace, workflowID string) + RequestCancelExternalWorkflow(namespace, workflowID, runID string, callback ResultHandler) + ExecuteChildWorkflow(params ExecuteWorkflowParams, callback ResultHandler, startedHandler func(r WorkflowExecution, e error)) + ExecuteNexusOperation(params executeNexusOperationParams, callback func(*commonpb.Payload, error), startedHandler func(opID string, e error)) int64 + RequestCancelNexusOperation(seq int64) + GetLogger() log.Logger + GetMetricsHandler() metrics.Handler + // Must be called before WorkflowDefinition.Execute returns + RegisterSignalHandler( + handler func(name string, input *commonpb.Payloads, header *commonpb.Header) error, + ) + SignalExternalWorkflow( + namespace string, + workflowID string, + runID string, + signalName string, + input *commonpb.Payloads, + arg interface{}, + header *commonpb.Header, + childWorkflowOnly bool, + callback ResultHandler, + ) + RegisterQueryHandler( + handler func(queryType string, queryArgs *commonpb.Payloads, header *commonpb.Header) (*commonpb.Payloads, error), + ) + RegisterUpdateHandler( + handler func(string, string, *commonpb.Payloads, *commonpb.Header, UpdateCallbacks), + ) + IsReplaying() bool + MutableSideEffect(id string, f func() interface{}, equals func(a, b interface{}) bool) converter.EncodedValue + GetDataConverter() converter.DataConverter + GetFailureConverter() converter.FailureConverter + AddSession(sessionInfo *SessionInfo) + RemoveSession(sessionID string) + GetContextPropagators() []ContextPropagator + UpsertSearchAttributes(attributes map[string]interface{}) error + UpsertTypedSearchAttributes(attributes SearchAttributes) error + UpsertMemo(memoMap map[string]interface{}) error + GetRegistry() *registry + // QueueUpdate request of type name + QueueUpdate(name string, f func()) + // HandleQueuedUpdates unblocks all queued updates of type name + HandleQueuedUpdates(name string) + // DrainUnhandledUpdates unblocks all updates, meant to be used to drain + // all unhandled updates at the end of a workflow task + // returns true if any update was unblocked + DrainUnhandledUpdates() bool + // TryUse returns true if this flag may currently be used. + TryUse(flag sdkFlag) bool + } + + // WorkflowDefinitionFactory factory for creating WorkflowDefinition instances. + WorkflowDefinitionFactory interface { + // NewWorkflowDefinition must return a new instance of WorkflowDefinition on each call. + NewWorkflowDefinition() WorkflowDefinition + } + + // WorkflowDefinition wraps the code that can execute a workflow. + WorkflowDefinition interface { + // Execute implementation must be asynchronous. + Execute(env WorkflowEnvironment, header *commonpb.Header, input *commonpb.Payloads) + // OnWorkflowTaskStarted is called for each non timed out startWorkflowTask event. + // Executed after all history events since the previous commands are applied to WorkflowDefinition + // Application level code must be executed from this function only. + // Execute call as well as callbacks called from WorkflowEnvironment functions can only schedule callbacks + // which can be executed from OnWorkflowTaskStarted(). + OnWorkflowTaskStarted(deadlockDetectionTimeout time.Duration) + // StackTrace of all coroutines owned by the Dispatcher instance. + StackTrace() string + // Close destroys all coroutines without waiting for their completion + Close() + } + + // baseWorkerOptions options to configure base worker. + baseWorkerOptions struct { + pollerCount int + pollerRate int + slotSupplier SlotSupplier + maxTaskPerSecond float64 + taskWorker taskPoller + workerType string + identity string + buildId string + logger log.Logger + stopTimeout time.Duration + fatalErrCb func(error) + userContextCancel context.CancelFunc + metricsHandler metrics.Handler + sessionTokenBucket *sessionTokenBucket + slotReservationData slotReservationData + } + + // baseWorker that wraps worker activities. + baseWorker struct { + options baseWorkerOptions + isWorkerStarted bool + stopCh chan struct{} // Channel used to stop the go routines. + stopWG sync.WaitGroup // The WaitGroup for stopping existing routines. + pollLimiter *rate.Limiter + taskLimiter *rate.Limiter + limiterContext context.Context + limiterContextCancel func() + retrier *backoff.ConcurrentRetrier // Service errors back off retrier + logger log.Logger + metricsHandler metrics.Handler + + slotSupplier *trackingSlotSupplier + taskQueueCh chan eagerOrPolledTask + eagerTaskQueueCh chan eagerTask + fatalErrCb func(error) + sessionTokenBucket *sessionTokenBucket + + lastPollTaskErrMessage string + lastPollTaskErrStarted time.Time + lastPollTaskErrLock sync.Mutex + } + + eagerOrPolledTask interface { + getTask() taskForWorker + getPermit() *SlotPermit + } + + polledTask struct { + task taskForWorker + permit *SlotPermit + } + + eagerTask struct { + // task to process. + task taskForWorker + permit *SlotPermit + } +) + +func (t *polledTask) getTask() taskForWorker { + return t.task +} +func (t *polledTask) getPermit() *SlotPermit { + return t.permit +} +func (t *eagerTask) getTask() taskForWorker { + return t.task +} +func (t *eagerTask) getPermit() *SlotPermit { + return t.permit +} + +// SetRetryLongPollGracePeriod sets the amount of time a long poller retries on +// fatal errors before it actually fails. For test use only, +// not safe to call with a running worker. +func SetRetryLongPollGracePeriod(period time.Duration) { + retryLongPollGracePeriod = period +} + +func getRetryLongPollGracePeriod() time.Duration { + return retryLongPollGracePeriod +} + +func createPollRetryPolicy() backoff.RetryPolicy { + policy := backoff.NewExponentialRetryPolicy(retryPollOperationInitialInterval) + policy.SetMaximumInterval(retryPollOperationMaxInterval) + + // NOTE: We don't use expiration interval since we don't use retries from retrier class. + // We use it to calculate next backoff. We have additional layer that is built on poller + // in the worker layer for to add some middleware for any poll retry that includes + // (a) rate limiting across pollers (b) back-off across pollers when server is busy + policy.SetExpirationInterval(retry.UnlimitedInterval) // We don't ever expire + return policy +} + +func createPollResourceExhaustedRetryPolicy() backoff.RetryPolicy { + policy := backoff.NewExponentialRetryPolicy(retryPollResourceExhaustedInitialInterval) + policy.SetMaximumInterval(retryPollResourceExhaustedMaxInterval) + policy.SetExpirationInterval(retry.UnlimitedInterval) + return policy +} + +func newBaseWorker( + options baseWorkerOptions, +) *baseWorker { + ctx, cancel := context.WithCancel(context.Background()) + logger := log.With(options.logger, tagWorkerType, options.workerType) + metricsHandler := options.metricsHandler.WithTags(metrics.WorkerTags(options.workerType)) + tss := newTrackingSlotSupplier(options.slotSupplier, trackingSlotSupplierOptions{ + logger: logger, + metricsHandler: metricsHandler, + workerBuildId: options.buildId, + workerIdentity: options.identity, + }) + bw := &baseWorker{ + options: options, + stopCh: make(chan struct{}), + taskLimiter: rate.NewLimiter(rate.Limit(options.maxTaskPerSecond), 1), + retrier: backoff.NewConcurrentRetrier(pollOperationRetryPolicy), + logger: logger, + metricsHandler: metricsHandler, + + slotSupplier: tss, + // No buffer, so pollers are only able to poll for new tasks after the previous one is + // dispatched. + taskQueueCh: make(chan eagerOrPolledTask), + // Allow enough capacity so that eager dispatch will not block. There's an upper limit of + // 2k pending activities so this channel never needs to be larger than that. + eagerTaskQueueCh: make(chan eagerTask, 2000), + fatalErrCb: options.fatalErrCb, + + limiterContext: ctx, + limiterContextCancel: cancel, + sessionTokenBucket: options.sessionTokenBucket, + } + // Set secondary retrier as resource exhausted + bw.retrier.SetSecondaryRetryPolicy(pollResourceExhaustedRetryPolicy) + if options.pollerRate > 0 { + bw.pollLimiter = rate.NewLimiter(rate.Limit(options.pollerRate), 1) + } + + return bw +} + +// Start starts a fixed set of routines to do the work. +func (bw *baseWorker) Start() { + if bw.isWorkerStarted { + return + } + + bw.metricsHandler.Counter(metrics.WorkerStartCounter).Inc(1) + + for i := 0; i < bw.options.pollerCount; i++ { + bw.stopWG.Add(1) + go bw.runPoller() + } + + bw.stopWG.Add(1) + go bw.runTaskDispatcher() + + bw.stopWG.Add(1) + go bw.runEagerTaskDispatcher() + + bw.isWorkerStarted = true + traceLog(func() { + bw.logger.Info("Started Worker", + "PollerCount", bw.options.pollerCount, + "MaxTaskPerSecond", bw.options.maxTaskPerSecond, + ) + }) +} + +func (bw *baseWorker) isStop() bool { + select { + case <-bw.stopCh: + return true + default: + return false + } +} + +func (bw *baseWorker) runPoller() { + defer bw.stopWG.Done() + bw.metricsHandler.Counter(metrics.PollerStartCounter).Inc(1) + + ctx, cancelfn := context.WithCancel(context.Background()) + defer cancelfn() + reserveChan := make(chan *SlotPermit) + + for { + bw.stopWG.Add(1) + go func() { + defer bw.stopWG.Done() + s, err := bw.slotSupplier.ReserveSlot(ctx, &bw.options.slotReservationData) + if err != nil { + if !errors.Is(err, context.Canceled) { + bw.logger.Error(fmt.Sprintf("Error while trying to reserve slot: %v", err)) + select { + case reserveChan <- nil: + case <-ctx.Done(): + return + } + } + return + } + select { + case reserveChan <- s: + case <-ctx.Done(): + bw.releaseSlot(s, SlotReleaseReasonUnused) + } + }() + + select { + case <-bw.stopCh: + return + case permit := <-reserveChan: + if permit == nil { // There was an error reserving a slot + // Avoid spamming reserve hard in the event it's constantly failing + if ctx.Err() == nil { + time.Sleep(time.Second) + } + continue + } + if bw.sessionTokenBucket != nil { + bw.sessionTokenBucket.waitForAvailableToken() + } + bw.pollTask(permit) + } + } +} + +func (bw *baseWorker) tryReserveSlot() *SlotPermit { + if bw.isStop() { + return nil + } + return bw.slotSupplier.TryReserveSlot(&bw.options.slotReservationData) +} + +func (bw *baseWorker) releaseSlot(permit *SlotPermit, reason SlotReleaseReason) { + bw.slotSupplier.ReleaseSlot(permit, reason) +} + +func (bw *baseWorker) pushEagerTask(task eagerTask) { + // Should always be non-blocking. Slots are reserved before requesting eager tasks. + bw.eagerTaskQueueCh <- task +} + +func (bw *baseWorker) processTaskAsync(eagerOrPolled eagerOrPolledTask) { + bw.stopWG.Add(1) + go func() { + defer bw.stopWG.Done() + + task := eagerOrPolled.getTask() + permit := eagerOrPolled.getPermit() + + if !task.isEmpty() { + bw.slotSupplier.MarkSlotUsed(permit) + } + + defer func() { + bw.releaseSlot(permit, SlotReleaseReasonTaskProcessed) + + if p := recover(); p != nil { + topLine := "base worker [panic]:" + st := getStackTraceRaw(topLine, 7, 0) + bw.logger.Error("Unhandled panic.", + "PanicError", fmt.Sprintf("%v", p), + "PanicStack", st) + } + }() + err := bw.options.taskWorker.ProcessTask(task) + if err != nil { + if isClientSideError(err) { + bw.logger.Info("Task processing failed with client side error", tagError, err) + } else { + bw.logger.Info("Task processing failed with error", tagError, err) + } + } + }() +} + +func (bw *baseWorker) runTaskDispatcher() { + defer bw.stopWG.Done() + + for { + // wait for new task or worker stop + select { + case <-bw.stopCh: + // Currently we can drop any tasks received when closing. + // https://github.com/temporalio/sdk-go/issues/1197 + return + case task := <-bw.taskQueueCh: + // for non-polled-task (local activity result as task or eager task), we don't need to rate limit + _, isPolledTask := task.(*polledTask) + if isPolledTask && bw.taskLimiter.Wait(bw.limiterContext) != nil { + if bw.isStop() { + bw.releaseSlot(task.getPermit(), SlotReleaseReasonUnused) + return + } + } + bw.processTaskAsync(task) + } + } +} + +func (bw *baseWorker) runEagerTaskDispatcher() { + defer bw.stopWG.Done() + for { + select { + case <-bw.stopCh: + // drain eager dispatch queue + for len(bw.eagerTaskQueueCh) > 0 { + eagerTask := <-bw.eagerTaskQueueCh + bw.processTaskAsync(&eagerTask) + } + return + case eagerTask := <-bw.eagerTaskQueueCh: + bw.processTaskAsync(&eagerTask) + } + } +} + +func (bw *baseWorker) pollTask(slotPermit *SlotPermit) { + var err error + var task taskForWorker + didSendTask := false + defer func() { + if !didSendTask { + bw.releaseSlot(slotPermit, SlotReleaseReasonUnused) + } + }() + + bw.retrier.Throttle(bw.stopCh) + if bw.pollLimiter == nil || bw.pollLimiter.Wait(bw.limiterContext) == nil { + task, err = bw.options.taskWorker.PollTask() + bw.logPollTaskError(err) + if err != nil { + // We retry "non retriable" errors while long polling for a while, because some proxies return + // unexpected values causing unnecessary downtime. + if isNonRetriableError(err) && bw.retrier.GetElapsedTime() > getRetryLongPollGracePeriod() { + bw.logger.Error("Worker received non-retriable error. Shutting down.", tagError, err) + if bw.fatalErrCb != nil { + bw.fatalErrCb(err) + } + return + } + // We use the secondary retrier on resource exhausted + _, resourceExhausted := err.(*serviceerror.ResourceExhausted) + bw.retrier.Failed(resourceExhausted) + } else { + bw.retrier.Succeeded() + } + } + + if task != nil { + select { + case bw.taskQueueCh <- &polledTask{task: task, permit: slotPermit}: + didSendTask = true + case <-bw.stopCh: + } + } +} + +func (bw *baseWorker) logPollTaskError(err error) { + // We do not want to log any errors after we were explicitly stopped + select { + case <-bw.stopCh: + return + default: + } + + bw.lastPollTaskErrLock.Lock() + defer bw.lastPollTaskErrLock.Unlock() + // No error means reset the message and time + if err == nil { + bw.lastPollTaskErrMessage = "" + bw.lastPollTaskErrStarted = time.Now() + return + } + // Log the error as warn if it doesn't match the last error seen or its over + // the time since + if err.Error() != bw.lastPollTaskErrMessage || time.Since(bw.lastPollTaskErrStarted) > lastPollTaskErrSuppressTime { + bw.logger.Warn("Failed to poll for task.", tagError, err) + bw.lastPollTaskErrMessage = err.Error() + bw.lastPollTaskErrStarted = time.Now() + } +} + +func isNonRetriableError(err error) bool { + if err == nil { + return false + } + switch err.(type) { + case *serviceerror.InvalidArgument, + *serviceerror.NamespaceNotFound, + *serviceerror.ClientVersionNotSupported: + return true + } + return false +} + +// Stop is a blocking call and cleans up all the resources associated with worker. +func (bw *baseWorker) Stop() { + if !bw.isWorkerStarted { + return + } + close(bw.stopCh) + bw.limiterContextCancel() + + if success := awaitWaitGroup(&bw.stopWG, bw.options.stopTimeout); !success { + traceLog(func() { + bw.logger.Info("Worker graceful stop timed out.", "Stop timeout", bw.options.stopTimeout) + }) + } + + // Close context + if bw.options.userContextCancel != nil { + bw.options.userContextCancel() + } + + bw.isWorkerStarted = false +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_worker_cache.go b/vendor/go.temporal.io/sdk/internal/internal_worker_cache.go new file mode 100644 index 00000000000..6e64127c91e --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_worker_cache.go @@ -0,0 +1,163 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "runtime" + "sync" + + "go.temporal.io/sdk/internal/common/cache" +) + +// A WorkerCache instance is held by each worker to hold cached data. The contents of this struct should always be +// pointers for any data shared with other workers, and owned values for any instance-specific caches. +type WorkerCache struct { + sharedCache *sharedWorkerCache +} + +// A container for data workers in this process may want to share with eachother +type sharedWorkerCache struct { + // Count of live workers + workerRefcount int + + // A cache workers can use to store workflow state. + workflowCache *cache.Cache + // Max size for the cache + maxWorkflowCacheSize int +} + +// A shared cache workers can use to store state. The cache is expected to be initialized with the first worker to be +// instantiated. IE: All workers have a pointer to it. The pointer itself is never made nil, but when the refcount +// reaches zero, the shared caches inside of it will be nilled out. Do not manipulate without holding +// sharedWorkerCacheLock +var sharedWorkerCachePtr = &sharedWorkerCache{} +var sharedWorkerCacheLock sync.Mutex + +// Must be set before spawning any workers +var desiredWorkflowCacheSize = defaultStickyCacheSize + +// SetStickyWorkflowCacheSize sets the cache size for sticky workflow cache. Sticky workflow execution is the affinity +// between workflow tasks of a specific workflow execution to a specific worker. The benefit of sticky execution is that +// the workflow does not have to reconstruct state by replaying history from the beginning. The cache is shared between +// workers running within same process. This must be called before any worker is started. If not called, the default +// size of 10K (which may change) will be used. +func SetStickyWorkflowCacheSize(cacheSize int) { + sharedWorkerCacheLock.Lock() + defer sharedWorkerCacheLock.Unlock() + desiredWorkflowCacheSize = cacheSize +} + +// PurgeStickyWorkflowCache resets the sticky workflow cache. This must be called only when all workers are stopped. +func PurgeStickyWorkflowCache() { + sharedWorkerCacheLock.Lock() + defer sharedWorkerCacheLock.Unlock() + + if sharedWorkerCachePtr.workflowCache != nil { + (*sharedWorkerCachePtr.workflowCache).Clear() + } +} + +// NewWorkerCache Creates a new WorkerCache, and increases workerRefcount by one. Instances of WorkerCache decrement the refcounter as +// a hook to runtime.SetFinalizer (ie: When they are freed by the GC). When there are no reachable instances of +// WorkerCache, shared caches will be cleared +func NewWorkerCache() *WorkerCache { + sharedWorkerCacheLock.Lock() + desiredWorkflowCacheSize := desiredWorkflowCacheSize + sharedWorkerCacheLock.Unlock() + + return newWorkerCache(sharedWorkerCachePtr, &sharedWorkerCacheLock, desiredWorkflowCacheSize) +} + +// This private version allows us to test functionality without affecting the global shared cache +func newWorkerCache(storeIn *sharedWorkerCache, lock *sync.Mutex, cacheSize int) *WorkerCache { + lock.Lock() + defer lock.Unlock() + + if storeIn == nil { + panic("Provided sharedWorkerCache pointer must not be nil") + } + + if storeIn.workerRefcount == 0 { + newcache := cache.New(cacheSize-1, &cache.Options{ + RemovedFunc: func(cachedEntity interface{}) { + wc := cachedEntity.(*workflowExecutionContextImpl) + wc.onEviction() + }, + }) + *storeIn = sharedWorkerCache{workflowCache: &newcache, workerRefcount: 0, maxWorkflowCacheSize: cacheSize} + } + storeIn.workerRefcount++ + newWorkerCache := WorkerCache{ + sharedCache: storeIn, + } + runtime.SetFinalizer(&newWorkerCache, func(wc *WorkerCache) { + wc.close(lock) + }) + return &newWorkerCache +} + +func (wc *WorkerCache) getWorkflowCache() cache.Cache { + return *wc.sharedCache.workflowCache +} + +func (wc *WorkerCache) close(lock *sync.Mutex) { + lock.Lock() + defer lock.Unlock() + + wc.sharedCache.workerRefcount-- + if wc.sharedCache.workerRefcount == 0 { + // Delete cache if no more outstanding references + wc.sharedCache.workflowCache = nil + } +} + +func (wc *WorkerCache) getWorkflowContext(runID string) *workflowExecutionContextImpl { + o := (*wc.sharedCache.workflowCache).Get(runID) + if o == nil { + return nil + } + wec := o.(*workflowExecutionContextImpl) + return wec +} + +func (wc *WorkerCache) putWorkflowContext(runID string, wec *workflowExecutionContextImpl) (*workflowExecutionContextImpl, error) { + existing, err := (*wc.sharedCache.workflowCache).PutIfNotExist(runID, wec) + if err != nil { + return nil, err + } + return existing.(*workflowExecutionContextImpl), nil +} + +func (wc *WorkerCache) removeWorkflowContext(runID string) { + (*wc.sharedCache.workflowCache).Delete(runID) +} + +// MaxWorkflowCacheSize returns the maximum allowed size of the sticky cache +func (wc *WorkerCache) MaxWorkflowCacheSize() int { + if wc == nil { + return desiredWorkflowCacheSize + } + return wc.sharedCache.maxWorkflowCacheSize +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_workflow.go b/vendor/go.temporal.io/sdk/internal/internal_workflow.go new file mode 100644 index 00000000000..7714472bfa7 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_workflow.go @@ -0,0 +1,1962 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +// All code in this file is private to the package. + +import ( + "bytes" + "errors" + "fmt" + "reflect" + "runtime" + "sort" + "strings" + "sync" + "sync/atomic" + "time" + "unicode" + + "golang.org/x/exp/slices" + + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/api/sdk/v1" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" +) + +const ( + defaultSignalChannelSize = 100000 // really large buffering size(100K) + defaultCoroutineExitTimeout = 100 * time.Millisecond + + panicIllegalAccessCoroutineState = "getState: illegal access from outside of workflow context" + unhandledUpdateWarningMessage = "[TMPRL1102] Workflow finished while update handlers are still running. This may have interrupted work that the" + + " update handler was doing, and the client that sent the update will receive a 'workflow execution" + + " already completed' RPCError instead of the update result. You can wait for all update" + + " handlers to complete by using `workflow.Await(ctx, func() bool { return workflow.AllHandlersFinished(ctx) })`. Alternatively, if both you and the clients sending the update" + + " are okay with interrupting running handlers when the workflow finishes, and causing clients to" + + " receive errors, then you can disable this warning via UnfinishedPolicy in UpdateHandlerOptions." +) + +type ( + syncWorkflowDefinition struct { + workflow workflow + dispatcher dispatcher + cancel CancelFunc + rootCtx Context + } + + workflowResult struct { + workflowResult *commonpb.Payloads + error error + } + + futureImpl struct { + value interface{} + err error + ready bool + channel *channelImpl + chained []asyncFuture // Futures that are chained to this one + } + + // Implements WaitGroup interface + waitGroupImpl struct { + n int // the number of coroutines to wait on + waiting bool // indicates whether WaitGroup.Wait() has been called yet for the WaitGroup + future Future // future to signal that all awaited members of the WaitGroup have completed + settable Settable // used to unblock the future when all coroutines have completed + } + + // Implements Mutex interface + mutexImpl struct { + locked bool + } + + // Implements Semaphore interface + semaphoreImpl struct { + size int64 + cur int64 + } + + // Dispatcher is a container of a set of coroutines. + dispatcher interface { + // ExecuteUntilAllBlocked executes coroutines one by one in deterministic order + // until all of them are completed or blocked on Channel or Selector or timeout is reached. + ExecuteUntilAllBlocked(deadlockDetectionTimeout time.Duration) (err error) + // IsDone returns true when all of coroutines are completed + IsDone() bool + IsClosed() bool + IsExecuting() bool + Close() // Destroys all coroutines without waiting for their completion + StackTrace() string // Stack trace of all coroutines owned by the Dispatcher instance + + // Create coroutine. To be called from within other coroutine. + // Used by the interceptors + NewCoroutine(ctx Context, name string, highPriority bool, f func(ctx Context)) Context + } + + // Workflow is an interface that any workflow should implement. + // Code of a workflow must be deterministic. It must use workflow.Channel, workflow.Selector, and workflow.Go instead of + // native channels, select and go. It also must not use range operation over map as it is randomized by go runtime. + // All time manipulation should use current time returned by GetTime(ctx) method. + // Note that workflow.Context is used instead of context.Context to avoid use of raw channels. + workflow interface { + Execute(ctx Context, input *commonpb.Payloads) (result *commonpb.Payloads, err error) + } + + sendCallback struct { + value interface{} + fn func() bool // false indicates that callback didn't accept the value + } + + receiveCallback struct { + // false result means that callback didn't accept the value and it is still up for delivery + fn func(v interface{}, more bool) bool + } + + channelImpl struct { + name string // human readable channel name + size int // Channel buffer size. 0 for non buffered. + buffer []interface{} // buffered messages + blockedSends []*sendCallback // puts waiting when buffer is full. + blockedReceives []*receiveCallback // receives waiting when no messages are available. + closed bool // true if channel is closed. + recValue *interface{} // Used only while receiving value, this is used as pre-fetch buffer value from the channel. + dataConverter converter.DataConverter // for decode data + env WorkflowEnvironment + } + + // Single case statement of the Select + selectCase struct { + channel *channelImpl // Channel of this case. + receiveFunc *func(c ReceiveChannel, more bool) // function to call when channel has a message. nil for send case. + + sendFunc *func() // function to call when channel accepted a message. nil for receive case. + sendValue *interface{} // value to send to the channel. Used only for send case. + future asyncFuture // Used for future case + futureFunc *func(f Future) // function to call when Future is ready + } + + // Implements Selector interface + selectorImpl struct { + name string + cases []*selectCase // cases that this select is comprised from + defaultFunc *func() // default case + } + + // unblockFunc is passed evaluated by a coroutine yield. When it returns false the yield returns to a caller. + // stackDepth is the depth of stack from the last blocking call relevant to user. + // Used to truncate internal stack frames from thread stack. + unblockFunc func(status string, stackDepth int) (keepBlocked bool) + + coroutineState struct { + name string + dispatcher *dispatcherImpl // dispatcher this context belongs to + aboutToBlock chan bool // used to notify dispatcher that coroutine that owns this context is about to block + unblock chan unblockFunc // used to notify coroutine that it should continue executing. + keptBlocked bool // true indicates that coroutine didn't make any progress since the last yield unblocking + closed atomic.Bool // indicates that owning coroutine has finished execution + blocked atomic.Bool + panicError error // non nil if coroutine had unhandled panic + } + + dispatcherImpl struct { + sequence int + channelSequence int // used to name channels + selectorSequence int // used to name channels + coroutines []*coroutineState + executing bool // currently running ExecuteUntilAllBlocked. Used to avoid recursive calls to it. + mutex sync.Mutex // used to synchronize executing + closed bool + interceptor WorkflowOutboundInterceptor + deadlockDetector *deadlockDetector + readOnly bool + // allBlockedCallback is called when all coroutines are blocked, + // returns true if the callback updated any coroutines state and there may be more work + allBlockedCallback func() bool + newEagerCoroutines []*coroutineState + } + + // WorkflowOptions options passed to the workflow function + // The current timeout resolution implementation is in seconds and uses math.Ceil() as the duration. But is + // subjected to change in the future. + WorkflowOptions struct { + TaskQueueName string + WorkflowExecutionTimeout time.Duration + WorkflowRunTimeout time.Duration + WorkflowTaskTimeout time.Duration + Namespace string + WorkflowID string + WaitForCancellation bool + WorkflowIDReusePolicy enumspb.WorkflowIdReusePolicy + WorkflowIDConflictPolicy enumspb.WorkflowIdConflictPolicy + DataConverter converter.DataConverter + RetryPolicy *commonpb.RetryPolicy + CronSchedule string + ContextPropagators []ContextPropagator + Memo map[string]interface{} + SearchAttributes map[string]interface{} + TypedSearchAttributes SearchAttributes + ParentClosePolicy enumspb.ParentClosePolicy + signalChannels map[string]Channel + requestedSignalChannels map[string]*requestedSignalChannel + queryHandlers map[string]*queryHandler + updateHandlers map[string]*updateHandler + // runningUpdatesHandles is a map of update handlers that are currently running. + runningUpdatesHandles map[string]UpdateInfo + VersioningIntent VersioningIntent + // TODO(cretz): Expose once https://github.com/temporalio/temporal/issues/6412 is fixed + staticSummary string + staticDetails string + // currentDetails is the user-set string returned on metadata query as + // WorkflowMetadata.current_details + currentDetails string + } + + // ExecuteWorkflowParams parameters of the workflow invocation + ExecuteWorkflowParams struct { + WorkflowOptions + WorkflowType *WorkflowType + Input *commonpb.Payloads + Header *commonpb.Header + attempt int32 // used by test framework to support child workflow retry + scheduledTime time.Time // used by test framework to support child workflow retry + lastCompletionResult *commonpb.Payloads // used by test framework to support cron + } + + // decodeFutureImpl + decodeFutureImpl struct { + *futureImpl + fn interface{} + } + + childWorkflowFutureImpl struct { + *decodeFutureImpl // for child workflow result + executionFuture *futureImpl // for child workflow execution future + } + + nexusOperationFutureImpl struct { + *decodeFutureImpl // for the result + executionFuture *futureImpl // for the NexusOperationExecution + } + + asyncFuture interface { + Future + // Used by selectorImpl + // If Future is ready returns its value immediately. + // If not registers callback which is called when it is ready. + GetAsync(callback *receiveCallback) (v interface{}, ok bool, err error) + + // Used by selectorImpl + RemoveReceiveCallback(callback *receiveCallback) + + // This future will added to list of dependency futures. + ChainFuture(f Future) + + // Gets the current value and error. + // Make sure this is called once the future is ready. + GetValueAndError() (v interface{}, err error) + + Set(value interface{}, err error) + } + + requestedSignalChannel struct { + options SignalChannelOptions + } + + queryHandler struct { + fn interface{} + queryType string + dataConverter converter.DataConverter + options QueryHandlerOptions + } + + // updateSchedulerImpl adapts the coro dispatcher to the UpdateScheduler interface + updateSchedulerImpl struct { + dispatcher dispatcher + } +) + +const ( + workflowEnvironmentContextKey = "workflowEnv" + workflowInterceptorContextKey = "workflowInterceptor" + localActivityFnContextKey = "localActivityFn" + workflowEnvInterceptorContextKey = "envInterceptor" + workflowResultContextKey = "workflowResult" + coroutinesContextKey = "coroutines" + workflowEnvOptionsContextKey = "wfEnvOptions" + updateInfoContextKey = "updateInfo" +) + +// Assert that structs do indeed implement the interfaces +var _ Channel = (*channelImpl)(nil) +var _ Selector = (*selectorImpl)(nil) +var _ WaitGroup = (*waitGroupImpl)(nil) +var _ dispatcher = (*dispatcherImpl)(nil) + +// 1MB buffer to fit combined stack trace of all active goroutines +var stackBuf [1024 * 1024]byte + +var ( + errCoroStackNotFound = errors.New("coroutine stack not found") + errStackTraceTruncated = errors.New("stack trace truncated: stackBuf is too small") +) + +// Pointer to pointer to workflow result +func getWorkflowResultPointerPointer(ctx Context) **workflowResult { + rpp := ctx.Value(workflowResultContextKey) + if rpp == nil { + panic("getWorkflowResultPointerPointer: Not a workflow context") + } + return rpp.(**workflowResult) +} + +func getWorkflowEnvironment(ctx Context) WorkflowEnvironment { + wc := ctx.Value(workflowEnvironmentContextKey) + if wc == nil { + panic("getWorkflowContext: Not a workflow context") + } + return wc.(WorkflowEnvironment) +} + +func getWorkflowEnvironmentInterceptor(ctx Context) *workflowEnvironmentInterceptor { + wc := ctx.Value(workflowEnvInterceptorContextKey) + if wc == nil { + panic("getWorkflowContext: Not a workflow context") + } + return wc.(*workflowEnvironmentInterceptor) +} + +type workflowEnvironmentInterceptor struct { + env WorkflowEnvironment + dispatcher dispatcher + inboundInterceptor WorkflowInboundInterceptor + fn interface{} + outboundInterceptor WorkflowOutboundInterceptor +} + +func (wc *workflowEnvironmentInterceptor) Go(ctx Context, name string, f func(ctx Context)) Context { + return wc.dispatcher.NewCoroutine(ctx, name, false, f) +} + +func getWorkflowOutboundInterceptor(ctx Context) WorkflowOutboundInterceptor { + wc := ctx.Value(workflowInterceptorContextKey) + if wc == nil { + panic("getWorkflowOutboundInterceptor: Not a workflow context") + } + return wc.(WorkflowOutboundInterceptor) +} + +func (f *futureImpl) Get(ctx Context, valuePtr interface{}) error { + assertNotInReadOnlyState(ctx) + more := f.channel.Receive(ctx, nil) + if more { + panic("not closed") + } + if !f.ready { + panic("not ready") + } + if f.err != nil || f.value == nil || valuePtr == nil { + return f.err + } + rf := reflect.ValueOf(valuePtr) + if rf.Type().Kind() != reflect.Ptr { + return errors.New("valuePtr parameter is not a pointer") + } + + if payload, ok := f.value.(*commonpb.Payloads); ok { + if _, ok2 := valuePtr.(**commonpb.Payloads); !ok2 { + if err := decodeArg(getDataConverterFromWorkflowContext(ctx), payload, valuePtr); err != nil { + return err + } + return f.err + } + } + + fv := reflect.ValueOf(f.value) + // If the value set was a pointer and is the same type as the wanted result, + // instead of panicking because it is not a pointer to a pointer, we will just + // set the pointer + if fv.Kind() == reflect.Ptr && fv.Type() == rf.Type() { + rf.Elem().Set(fv.Elem()) + } else { + rf.Elem().Set(fv) + } + return f.err +} + +// Used by selectorImpl +// If Future is ready returns its value immediately. +// If not registers callback which is called when it is ready. +func (f *futureImpl) GetAsync(callback *receiveCallback) (v interface{}, ok bool, err error) { + _, _, more := f.channel.receiveAsyncImpl(callback) + // Future uses Channel.Close to indicate that it is ready. + // So more being true (channel is still open) indicates future is not ready. + if more { + return nil, false, nil + } + if !f.ready { + panic("not ready") + } + return f.value, true, f.err +} + +// RemoveReceiveCallback removes the callback from future's channel to avoid closure leak. +// Used by selectorImpl +func (f *futureImpl) RemoveReceiveCallback(callback *receiveCallback) { + f.channel.removeReceiveCallback(callback) +} + +func (f *futureImpl) IsReady() bool { + return f.ready +} + +func (f *futureImpl) Set(value interface{}, err error) { + if f.ready { + panic("already set") + } + f.value = value + f.err = err + f.ready = true + f.channel.Close() + for _, ch := range f.chained { + ch.Set(f.value, f.err) + } +} + +func (f *futureImpl) SetValue(value interface{}) { + if f.ready { + panic("already set") + } + f.Set(value, nil) +} + +func (f *futureImpl) SetError(err error) { + if f.ready { + panic("already set") + } + f.Set(nil, err) +} + +func (f *futureImpl) Chain(future Future) { + if f.ready { + panic("already set") + } + + ch, ok := future.(asyncFuture) + if !ok { + panic("cannot chain Future that wasn't created with workflow.NewFuture") + } + if !ch.IsReady() { + ch.ChainFuture(f) + return + } + val, err := ch.GetValueAndError() + f.value = val + f.err = err + f.ready = true +} + +func (f *futureImpl) ChainFuture(future Future) { + f.chained = append(f.chained, future.(asyncFuture)) +} + +func (f *futureImpl) GetValueAndError() (interface{}, error) { + return f.value, f.err +} + +func (f *childWorkflowFutureImpl) GetChildWorkflowExecution() Future { + return f.executionFuture +} + +func (f *childWorkflowFutureImpl) SignalChildWorkflow(ctx Context, signalName string, data interface{}) Future { + assertNotInReadOnlyState(ctx) + var childExec WorkflowExecution + if err := f.GetChildWorkflowExecution().Get(ctx, &childExec); err != nil { + return f.GetChildWorkflowExecution() + } + + i := getWorkflowOutboundInterceptor(ctx) + // Put header on context before executing + ctx = workflowContextWithNewHeader(ctx) + return i.SignalChildWorkflow(ctx, childExec.ID, signalName, data) +} + +func (f *nexusOperationFutureImpl) GetNexusOperationExecution() Future { + return f.executionFuture +} + +func newWorkflowContext( + env WorkflowEnvironment, + interceptors []WorkerInterceptor, +) (*workflowEnvironmentInterceptor, Context, error) { + // Create context with default values + ctx := WithValue(background, workflowEnvironmentContextKey, env) + var resultPtr *workflowResult + ctx = WithValue(ctx, workflowResultContextKey, &resultPtr) + info := env.WorkflowInfo() + ctx = WithWorkflowNamespace(ctx, info.Namespace) + ctx = WithWorkflowTaskQueue(ctx, info.TaskQueueName) + getWorkflowEnvOptions(ctx).WorkflowExecutionTimeout = info.WorkflowExecutionTimeout + ctx = WithWorkflowRunTimeout(ctx, info.WorkflowRunTimeout) + ctx = WithWorkflowTaskTimeout(ctx, info.WorkflowTaskTimeout) + ctx = WithTaskQueue(ctx, info.TaskQueueName) + ctx = WithDataConverter(ctx, env.GetDataConverter()) + ctx = withContextPropagators(ctx, env.GetContextPropagators()) + getActivityOptions(ctx).OriginalTaskQueueName = info.TaskQueueName + + // Create interceptor and put it on context as inbound and put it on context + // as the default outbound interceptor before init + envInterceptor := &workflowEnvironmentInterceptor{env: env} + envInterceptor.inboundInterceptor = envInterceptor + envInterceptor.outboundInterceptor = envInterceptor + ctx = WithValue(ctx, workflowEnvInterceptorContextKey, envInterceptor) + ctx = WithValue(ctx, workflowInterceptorContextKey, envInterceptor.outboundInterceptor) + + // Intercept, run init, and put the new outbound interceptor on the context + for i := len(interceptors) - 1; i >= 0; i-- { + envInterceptor.inboundInterceptor = interceptors[i].InterceptWorkflow(ctx, envInterceptor.inboundInterceptor) + } + err := envInterceptor.inboundInterceptor.Init(envInterceptor) + if err != nil { + return nil, nil, err + } + ctx = WithValue(ctx, workflowInterceptorContextKey, envInterceptor.outboundInterceptor) + + return envInterceptor, ctx, nil +} + +func (d *syncWorkflowDefinition) Execute(env WorkflowEnvironment, header *commonpb.Header, input *commonpb.Payloads) { + envInterceptor, rootCtx, err := newWorkflowContext(env, env.GetRegistry().interceptors) + if err != nil { + panic(err) + } + dispatcher, rootCtx := newDispatcher( + rootCtx, + envInterceptor, + func(ctx Context) { + r := &workflowResult{} + + // We want to execute the user workflow definition from the first workflow task started, + // so they can see everything before that. Here we would have all initialization done, hence + // we are yielding. + state := getState(d.rootCtx) + state.yield("yield before executing to setup state") + state.unblocked() + + r.workflowResult, r.error = d.workflow.Execute(d.rootCtx, input) + rpp := getWorkflowResultPointerPointer(ctx) + *rpp = r + }, getWorkflowEnvironment(rootCtx).DrainUnhandledUpdates) + + // set the information from the headers that is to be propagated in the workflow context + rootCtx, err = workflowContextWithHeaderPropagated(rootCtx, header, env.GetContextPropagators()) + if err != nil { + panic(err) + } + + d.rootCtx, d.cancel = WithCancel(rootCtx) + d.dispatcher = dispatcher + envInterceptor.dispatcher = dispatcher + + getWorkflowEnvironment(d.rootCtx).RegisterCancelHandler(func() { + // It is ok to call this method multiple times. + // it doesn't do anything new, the context remains canceled. + d.cancel() + }) + + getWorkflowEnvironment(d.rootCtx).RegisterSignalHandler( + func(name string, input *commonpb.Payloads, header *commonpb.Header) error { + // Put the header on context + rootCtx, err := workflowContextWithHeaderPropagated(d.rootCtx, header, env.GetContextPropagators()) + if err != nil { + return err + } + return envInterceptor.inboundInterceptor.HandleSignal(rootCtx, &HandleSignalInput{SignalName: name, Arg: input}) + }, + ) + + getWorkflowEnvironment(d.rootCtx).RegisterUpdateHandler( + func(name string, id string, serializedArgs *commonpb.Payloads, header *commonpb.Header, callbacks UpdateCallbacks) { + defaultUpdateHandler(d.rootCtx, name, id, serializedArgs, header, callbacks, updateSchedulerImpl{d.dispatcher}) + }) + + getWorkflowEnvironment(d.rootCtx).RegisterQueryHandler( + func(queryType string, queryArgs *commonpb.Payloads, header *commonpb.Header) (*commonpb.Payloads, error) { + // Put the header on context if server supports it + rootCtx, err := workflowContextWithHeaderPropagated(d.rootCtx, header, env.GetContextPropagators()) + if err != nil { + return nil, err + } + + // As a special case, we handle __temporal_workflow_metadata query + // here instead of in workflowExecutionEventHandlerImpl.ProcessQuery + // because we need the context environment to do so. + if queryType == QueryTypeWorkflowMetadata { + if result, err := getWorkflowMetadata(rootCtx); err != nil { + return nil, err + } else { + return encodeArg(getDataConverterFromWorkflowContext(rootCtx), result) + } + } + + eo := getWorkflowEnvOptions(rootCtx) + // A handler must be present since it is needed for argument decoding, + // even if the interceptor intercepts query handling + handler, ok := eo.queryHandlers[queryType] + if !ok { + keys := []string{QueryTypeStackTrace, QueryTypeOpenSessions, QueryTypeWorkflowMetadata} + for k := range eo.queryHandlers { + keys = append(keys, k) + } + return nil, fmt.Errorf("unknown queryType %v. KnownQueryTypes=%v", queryType, keys) + } + + // Decode the arguments + args, err := decodeArgsToRawValues(handler.dataConverter, reflect.TypeOf(handler.fn), queryArgs) + if err != nil { + return nil, fmt.Errorf("unable to decode the input for queryType: %v, with error: %w", handler.queryType, err) + } + + // Invoke + result, err := envInterceptor.inboundInterceptor.HandleQuery( + rootCtx, + &HandleQueryInput{QueryType: queryType, Args: args}, + ) + + // Encode the result + var serializedResult *commonpb.Payloads + if err == nil { + serializedResult, err = encodeArg(handler.dataConverter, result) + } + return serializedResult, err + }, + ) +} + +func (d *syncWorkflowDefinition) OnWorkflowTaskStarted(deadlockDetectionTimeout time.Duration) { + executeDispatcher(d.rootCtx, d.dispatcher, deadlockDetectionTimeout) +} + +func (d *syncWorkflowDefinition) StackTrace() string { + return d.dispatcher.StackTrace() +} + +func (d *syncWorkflowDefinition) Close() { + if d.dispatcher != nil { + d.dispatcher.Close() + } +} + +// NewDispatcher creates a new Dispatcher instance with a root coroutine function. +// Context passed to the root function is child of the passed rootCtx. +// This way rootCtx can be used to pass values to the coroutine code. +func newDispatcher(rootCtx Context, interceptor *workflowEnvironmentInterceptor, root func(ctx Context), allBlockedCallback func() bool) (*dispatcherImpl, Context) { + result := &dispatcherImpl{ + interceptor: interceptor.outboundInterceptor, + deadlockDetector: newDeadlockDetector(), + allBlockedCallback: allBlockedCallback, + } + interceptor.dispatcher = result + ctxWithState := result.interceptor.Go(rootCtx, "root", root) + return result, ctxWithState +} + +// executeDispatcher executed coroutines in the calling thread and calls workflow completion callbacks +// if root workflow function returned +func executeDispatcher(ctx Context, dispatcher dispatcher, timeout time.Duration) { + env := getWorkflowEnvironment(ctx) + panicErr := dispatcher.ExecuteUntilAllBlocked(timeout) + if panicErr != nil { + env.Complete(nil, panicErr) + return + } + + rp := *getWorkflowResultPointerPointer(ctx) + if rp == nil { + // Result is not set, so workflow is still executing + return + } + + weo := getWorkflowEnvOptions(ctx) + us := weo.getUnhandledSignalNames() + if len(us) > 0 { + env.GetLogger().Warn("Workflow has unhandled signals", "SignalNames", us) + } + // Warn if there are any update handlers still running + type warnUpdate struct { + Name string `json:"name"` + ID string `json:"id"` + } + var updatesToWarn []warnUpdate + for _, info := range weo.getRunningUpdateHandles() { + if weo.updateHandlers[info.Name].unfinishedPolicy == HandlerUnfinishedPolicyWarnAndAbandon { + updatesToWarn = append(updatesToWarn, warnUpdate{ + Name: info.Name, + ID: info.ID, + }) + } + } + + // Verify that the workflow did not fail. If it did we will not warn about unhandled updates. + var canceledErr *CanceledError + var contErr *ContinueAsNewError + if len(updatesToWarn) > 0 && (rp.error == nil || errors.As(rp.error, &canceledErr) || errors.As(rp.error, &contErr)) { + env.GetLogger().Warn(unhandledUpdateWarningMessage, "Updates", updatesToWarn) + } + + env.Complete(rp.workflowResult, rp.error) +} + +// For troubleshooting stack pretty printing only. +// Set to true to see full stack trace that includes framework methods. +const disableCleanStackTraces = false + +func getState(ctx Context) *coroutineState { + s := ctx.Value(coroutinesContextKey) + if s == nil { + panic("getState: not workflow context") + } + state := s.(*coroutineState) + if !state.dispatcher.IsExecuting() { + panic(panicIllegalAccessCoroutineState) + } + return state +} + +func assertNotInReadOnlyState(ctx Context) { + state := getState(ctx) + // use the dispatcher state instead of the coroutine state because contexts can be + // shared + if state.dispatcher.getIsReadOnly() { + panic(panicIllegalAccessCoroutineState) + } +} + +func assertNotInReadOnlyStateCancellation(ctx Context) { + s := ctx.Value(coroutinesContextKey) + if s == nil { + panic("assertNotInReadOnlyStateCtxCancellation: not workflow context") + } + state := s.(*coroutineState) + // For cancellation the dispatcher may not be running because workflow cancellation + // is sent outside of the dispatchers loop. + if state.dispatcher.IsClosed() { + panic(panicIllegalAccessCoroutineState) + } + // use the dispatcher state instead of the coroutine state because contexts can be + // shared + if state.dispatcher.getIsReadOnly() { + panic(panicIllegalAccessCoroutineState) + } +} + +func getStateIfRunning(ctx Context) *coroutineState { + if ctx == nil { + return nil + } + s := ctx.Value(coroutinesContextKey) + if s == nil { + return nil + } + state := s.(*coroutineState) + if !state.dispatcher.IsExecuting() { + return nil + } + return state +} + +func (c *channelImpl) Name() string { + return c.name +} + +func (c *channelImpl) CanReceiveWithoutBlocking() bool { + return c.recValue != nil || len(c.buffer) > 0 || len(c.blockedSends) > 0 || c.closed +} + +func (c *channelImpl) CanSendWithoutBlocking() bool { + return len(c.buffer) < c.size || len(c.blockedReceives) > 0 +} + +func (c *channelImpl) Receive(ctx Context, valuePtr interface{}) (more bool) { + assertNotInReadOnlyState(ctx) + state := getState(ctx) + hasResult := false + var result interface{} + callback := &receiveCallback{ + fn: func(v interface{}, m bool) bool { + result = v + hasResult = true + more = m + return true + }, + } + + for { + hasResult = false + v, ok, m := c.receiveAsyncImpl(callback) + + if !ok && !m { // channel closed and empty + return m + } + + if ok || !m { + err := c.assignValue(v, valuePtr) + if err == nil { + state.unblocked() + return m + } + continue // corrupt signal. Drop and reset process + } + for { + if hasResult { + err := c.assignValue(result, valuePtr) + if err == nil { + state.unblocked() + return more + } + break // Corrupt signal. Drop and reset process. + } + state.yield("blocked on " + c.name + ".Receive") + } + } + +} + +func (c *channelImpl) ReceiveWithTimeout(ctx Context, timeout time.Duration, valuePtr interface{}) (ok, more bool) { + okAwait, err := AwaitWithTimeout(ctx, timeout, func() bool { return c.Len() > 0 }) + if err != nil { // context canceled + return false, true + } + if !okAwait { // timed out + return false, true + } + ok, more = c.ReceiveAsyncWithMoreFlag(valuePtr) + if !ok { + panic("unexpected empty channel") + } + return true, more +} + +func (c *channelImpl) ReceiveAsync(valuePtr interface{}) (ok bool) { + ok, _ = c.ReceiveAsyncWithMoreFlag(valuePtr) + return ok +} + +func (c *channelImpl) ReceiveAsyncWithMoreFlag(valuePtr interface{}) (ok bool, more bool) { + for { + v, ok, more := c.receiveAsyncImpl(nil) + if !ok && !more { // channel closed and empty + return ok, more + } + + err := c.assignValue(v, valuePtr) + if err != nil { + continue + // keep consuming until a good signal is hit or channel is drained + } + return ok, more + } +} + +func (c *channelImpl) Len() int { + result := len(c.buffer) + len(c.blockedSends) + if c.recValue != nil { + result = result + 1 + } + return result +} + +// ok = true means that value was received +// more = true means that channel is not closed and more deliveries are possible +func (c *channelImpl) receiveAsyncImpl(callback *receiveCallback) (v interface{}, ok bool, more bool) { + if c.recValue != nil { + r := *c.recValue + c.recValue = nil + return r, true, true + } + if len(c.buffer) > 0 { + r := c.buffer[0] + c.buffer[0] = nil + c.buffer = c.buffer[1:] + + // Move blocked sends into buffer + for len(c.blockedSends) > 0 { + b := c.blockedSends[0] + c.blockedSends[0] = nil + c.blockedSends = c.blockedSends[1:] + if b.fn() { + c.buffer = append(c.buffer, b.value) + break + } + } + + return r, true, true + } + if c.closed { + return nil, false, false + } + for len(c.blockedSends) > 0 { + b := c.blockedSends[0] + c.blockedSends[0] = nil + c.blockedSends = c.blockedSends[1:] + if b.fn() { + return b.value, true, true + } + } + if callback != nil { + c.blockedReceives = append(c.blockedReceives, callback) + } + return nil, false, true +} + +func (c *channelImpl) removeReceiveCallback(callback *receiveCallback) { + for i, blockedCallback := range c.blockedReceives { + if callback == blockedCallback { + c.blockedReceives = append(c.blockedReceives[:i], c.blockedReceives[i+1:]...) + break + } + } +} + +func (c *channelImpl) removeSendCallback(callback *sendCallback) { + for i, blockedCallback := range c.blockedSends { + if callback == blockedCallback { + c.blockedSends = append(c.blockedSends[:i], c.blockedSends[i+1:]...) + break + } + } +} + +func (c *channelImpl) Send(ctx Context, v interface{}) { + state := getState(ctx) + valueConsumed := false + callback := &sendCallback{ + value: v, + fn: func() bool { + valueConsumed = true + return true + }, + } + ok := c.sendAsyncImpl(v, callback) + if ok { + state.unblocked() + return + } + for { + if valueConsumed { + state.unblocked() + return + } + + // Check for closed in the loop as close can be called when send is blocked + if c.closed { + panic("Closed channel") + } + state.yield("blocked on " + c.name + ".Send") + } +} + +func (c *channelImpl) SendAsync(v interface{}) (ok bool) { + return c.sendAsyncImpl(v, nil) +} + +func (c *channelImpl) sendAsyncImpl(v interface{}, pair *sendCallback) (ok bool) { + if c.closed { + panic("Closed channel") + } + for len(c.blockedReceives) > 0 { + blockedGet := c.blockedReceives[0].fn + c.blockedReceives[0] = nil + c.blockedReceives = c.blockedReceives[1:] + // false from callback indicates that value wasn't consumed + if blockedGet(v, true) { + return true + } + } + if len(c.buffer) < c.size { + c.buffer = append(c.buffer, v) + return true + } + if pair != nil { + c.blockedSends = append(c.blockedSends, pair) + } + return false +} + +func (c *channelImpl) Close() { + c.closed = true + // Use a copy of blockedReceives for iteration as invoking callback could result in modification + copy := append(c.blockedReceives[:0:0], c.blockedReceives...) + for _, callback := range copy { + callback.fn(nil, false) + } + // All blocked sends are going to panic +} + +// Takes a value and assigns that 'to' value. logs a metric if it is unable to deserialize +func (c *channelImpl) assignValue(from interface{}, to interface{}) error { + err := decodeAndAssignValue(c.dataConverter, from, to) + // add to metrics + if err != nil { + c.env.GetLogger().Error(fmt.Sprintf("Deserialization error. Corrupted signal received on channel %s.", c.name), tagError, err) + c.env.GetMetricsHandler().Counter(metrics.CorruptedSignalsCounter).Inc(1) + } + return err +} + +// initialYield called at the beginning of the coroutine execution +// stackDepth is the depth of top of the stack to omit when stack trace is generated +// to hide frames internal to the framework. +func (s *coroutineState) initialYield(stackDepth int, status string) { + if s.blocked.Swap(true) { + panic("trying to block on coroutine which is already blocked, most likely a wrong Context is used to do blocking" + + " call (like Future.Get() or Channel.Receive()") + } + keepBlocked := true + for keepBlocked { + f := <-s.unblock + keepBlocked = f(status, stackDepth+1) + } + s.blocked.Swap(false) +} + +// yield indicates that coroutine cannot make progress and should sleep +// this call blocks +func (s *coroutineState) yield(status string) { + s.aboutToBlock <- true + s.initialYield(3, status) // omit three levels of stack. To adjust change to 0 and count the lines to remove. + s.keptBlocked = true +} + +func getStackTrace(coroutineName, status string, stackDepth int) string { + top := fmt.Sprintf("coroutine %s [%s]:", coroutineName, status) + // Omit top stackDepth frames + top status line. + // Omit bottom two frames which is wrapping of coroutine in a goroutine. + return getStackTraceRaw(top, stackDepth*2+1, 4) +} + +func getStackTraceRaw(top string, omitTop, omitBottom int) string { + stack := stackBuf[:runtime.Stack(stackBuf[:], false)] + outStack := filterStackTrace(string(stack), omitTop, omitBottom) + return strings.Join([]string{top, outStack}, "\n") +} + +func filterStackTrace(stack string, omitTop, omitBottom int) string { + stack = strings.TrimRightFunc(stack, unicode.IsSpace) + if disableCleanStackTraces { + return stack + } + + lines := strings.Split(stack, "\n") + omitEnd := len(lines) - omitBottom + // If the start is after the end, the depth was invalid originally so return + // the entire raw stack + if omitTop > omitEnd { + return stack + } + return strings.Join(lines[omitTop:omitEnd], "\n") +} + +func getCoroStackTrace(crt *coroutineState, status string, stackDepth int) (string, error) { + // Can't dump goroutines selectively :( + // Instead, we identify a coroutine's stack trace by the *coroutineState pointer address + // in its function arguments. To avoid false positives, we also match on the fixed + // member function name. + stacks := stackBuf[:runtime.Stack(stackBuf[:], true)] + needle := []byte(fmt.Sprintf("/internal.(*coroutineState).run(%p,", crt)) + idx := bytes.Index(stacks, needle) + if idx == -1 { + if len(stacks) == len(stackBuf) { + return "", fmt.Errorf("coroutine not found: %w", errStackTraceTruncated) + } + // NOTE: This could happen if coroutineState is moved between runtime.Stack(...) + // and formatting needle. However, Go's GC is currently non-moving. + return "", errCoroStackNotFound + } + + // coroStack spans from the stackDelim before idx to the stackDelim after idx + stackDelim := []byte("\n\n") + coroStack := stacks + if start := bytes.LastIndex(stacks[:idx], stackDelim); start != -1 { + start += len(stackDelim) // skip over delimiter + coroStack = stacks[start:] + } + coroStack, _, _ = bytes.Cut(coroStack, stackDelim) + + // Omit top stackDepth frames + top status line. + // Omit bottom two frames which is wrapping of coroutine in a goroutine. + outStack := filterStackTrace(string(coroStack), stackDepth*2+1, 4) + return fmt.Sprintf("coroutine %s [%s]:\n%s", crt.name, status, outStack), nil +} + +// unblocked is called by coroutine to indicate that since the last time yield was unblocked channel or select +// where unblocked versus calling yield again after checking their condition +func (s *coroutineState) unblocked() { + s.keptBlocked = false +} + +func (s *coroutineState) call(timeout time.Duration) { + s.unblock <- func(status string, stackDepth int) bool { + return false // unblock + } + + // Defaults are populated in the worker options during worker startup, but test environment + // may have no default value for the deadlock detection timeout, so we also need to set it here for + // backwards compatibility. + if timeout == 0 { + timeout = defaultDeadlockDetectionTimeout + if debugMode { + timeout = unlimitedDeadlockDetectionTimeout + } + } + deadlockTicker := s.dispatcher.deadlockDetector.begin(timeout) + defer deadlockTicker.end() + + select { + case <-s.aboutToBlock: + case <-deadlockTicker.reached(): + // Use workflowPanicError since this used to call panic(msg) + st, err := getCoroStackTrace(s, "running", 0) + if err != nil { + st = fmt.Sprintf("<%s>", err) + } + msg := fmt.Sprintf("[TMPRL1101] Potential deadlock detected: "+ + "workflow goroutine %q didn't yield for over a second", s.name) + s.closed.Store(true) + s.panicError = newWorkflowPanicError(msg, st) + } +} + +func (s *coroutineState) close() { + s.closed.Store(true) + s.aboutToBlock <- true +} + +// exit tries to run Goexit on the coroutine and wait for it to exit +// within timeout. +func (s *coroutineState) exit(timeout time.Duration) { + if !s.closed.Load() { + s.unblock <- func(status string, stackDepth int) bool { + runtime.Goexit() + return true + } + + timer := time.NewTimer(timeout) + defer timer.Stop() + + select { + case <-s.aboutToBlock: + case <-timer.C: + } + } +} + +func (s *coroutineState) stackTrace() string { + if s.closed.Load() { + return "" + } + stackCh := make(chan string, 1) + s.unblock <- func(status string, stackDepth int) bool { + stackCh <- getStackTrace(s.name, status, stackDepth+2) + return true + } + return <-stackCh +} + +func (s *coroutineState) run(ctx Context, f func(ctx Context)) { + defer runtime.KeepAlive(&s) // keep receiver argument alive for getCoroStackTrace + defer s.close() + defer func() { + if r := recover(); r != nil { + st := getStackTrace(s.name, "panic", 4) + s.panicError = newWorkflowPanicError(r, st) + } + }() + s.initialYield(1, "") + f(ctx) +} + +func (d *dispatcherImpl) NewCoroutine(ctx Context, name string, highPriority bool, f func(ctx Context)) Context { + if name == "" { + name = fmt.Sprintf("%v", d.sequence+1) + } + state := d.newState(name, highPriority) + spawned := WithValue(ctx, coroutinesContextKey, state) + go state.run(spawned, f) + return spawned +} + +func (d *dispatcherImpl) newState(name string, highPriority bool) *coroutineState { + c := &coroutineState{ + name: name, + dispatcher: d, + aboutToBlock: make(chan bool, 1), + unblock: make(chan unblockFunc), + } + d.sequence++ + if highPriority { + // Update requests need to be added to the front of the dispatchers coroutine list so they + // are handled before the root coroutine. + d.newEagerCoroutines = append(d.newEagerCoroutines, c) + } else { + d.coroutines = append(d.coroutines, c) + } + return c +} + +func (d *dispatcherImpl) IsClosed() bool { + d.mutex.Lock() + defer d.mutex.Unlock() + return d.closed +} + +func (d *dispatcherImpl) ExecuteUntilAllBlocked(deadlockDetectionTimeout time.Duration) (err error) { + d.mutex.Lock() + if d.closed { + d.mutex.Unlock() + panic("dispatcher is closed") + } + if d.executing { + d.mutex.Unlock() + panic("call to ExecuteUntilAllBlocked (possibly from a coroutine) while it is already running") + } + d.executing = true + d.mutex.Unlock() + defer func() { + d.mutex.Lock() + d.executing = false + d.mutex.Unlock() + }() + allBlocked := false + // Keep executing until at least one goroutine made some progress + for !allBlocked || d.allBlockedCallback() { + d.coroutines = append(d.newEagerCoroutines, d.coroutines...) + d.newEagerCoroutines = nil + // Give every coroutine chance to execute removing closed ones + allBlocked = true + lastSequence := d.sequence + for i := 0; i < len(d.coroutines); i++ { + c := d.coroutines[i] + if !c.closed.Load() { + // TODO: Support handling of panic in a coroutine by dispatcher. + // TODO: Dump all outstanding coroutines if one of them panics + c.call(deadlockDetectionTimeout) + } + // c.call() can close the context so check again + if c.closed.Load() { + // remove the closed one from the slice + d.coroutines = append(d.coroutines[:i], + d.coroutines[i+1:]...) + i-- + if c.panicError != nil { + return c.panicError + } + allBlocked = false + + } else { + allBlocked = allBlocked && (c.keptBlocked || c.closed.Load()) + } + // If any eager coroutines were created by the last coroutine we + // need to schedule them now. + if len(d.newEagerCoroutines) > 0 { + d.coroutines = slices.Insert(d.coroutines, i+1, d.newEagerCoroutines...) + d.newEagerCoroutines = nil + allBlocked = false + } + } + // Set allBlocked to false if new coroutines where created + allBlocked = allBlocked && lastSequence == d.sequence + } + return nil +} + +func (d *dispatcherImpl) IsDone() bool { + d.mutex.Lock() + defer d.mutex.Unlock() + return len(d.coroutines) == 0 +} + +func (d *dispatcherImpl) IsExecuting() bool { + d.mutex.Lock() + defer d.mutex.Unlock() + return d.executing +} + +func (d *dispatcherImpl) getIsReadOnly() bool { + d.mutex.Lock() + defer d.mutex.Unlock() + return d.readOnly +} + +func (d *dispatcherImpl) setIsReadOnly(readOnly bool) { + d.mutex.Lock() + defer d.mutex.Unlock() + d.readOnly = readOnly +} + +func (d *dispatcherImpl) Close() { + d.mutex.Lock() + if d.closed { + d.mutex.Unlock() + return + } + d.closed = true + d.mutex.Unlock() + // We need to exit the coroutines in a separate goroutine because: + // * The coroutine may be stuck and won't respond to the exit request. + // * On exit the coroutines defers will still run and that may block. + go func() { + for _, c := range d.coroutines { + c.exit(defaultCoroutineExitTimeout) + } + }() +} + +func (d *dispatcherImpl) StackTrace() string { + var result string + for i := 0; i < len(d.coroutines); i++ { + c := d.coroutines[i] + if !c.closed.Load() { + if len(result) > 0 { + result += "\n\n" + } + result += c.stackTrace() + } + } + return result +} + +func (s *selectorImpl) AddReceive(c ReceiveChannel, f func(c ReceiveChannel, more bool)) Selector { + s.cases = append(s.cases, &selectCase{channel: c.(*channelImpl), receiveFunc: &f}) + return s +} + +func (s *selectorImpl) AddSend(c SendChannel, v interface{}, f func()) Selector { + s.cases = append(s.cases, &selectCase{channel: c.(*channelImpl), sendFunc: &f, sendValue: &v}) + return s +} + +func (s *selectorImpl) AddFuture(future Future, f func(future Future)) Selector { + asyncF, ok := future.(asyncFuture) + if !ok { + panic("cannot chain Future that wasn't created with workflow.NewFuture") + } + s.cases = append(s.cases, &selectCase{future: asyncF, futureFunc: &f}) + return s +} + +func (s *selectorImpl) AddDefault(f func()) { + s.defaultFunc = &f +} + +func (s *selectorImpl) HasPending() bool { + for _, pair := range s.cases { + if pair.receiveFunc != nil && pair.channel.CanReceiveWithoutBlocking() { + return true + } else if pair.sendFunc != nil && pair.channel.CanSendWithoutBlocking() { + return true + } else if pair.futureFunc != nil && pair.future.IsReady() { + return true + } + } + return false +} + +func (s *selectorImpl) Select(ctx Context) { + assertNotInReadOnlyState(ctx) + state := getState(ctx) + var readyBranch func() + var cleanups []func() + defer func() { + for _, c := range cleanups { + c() + } + }() + + for _, pair := range s.cases { + if pair.receiveFunc != nil { + f := *pair.receiveFunc + c := pair.channel + callback := &receiveCallback{ + fn: func(v interface{}, more bool) bool { + if readyBranch != nil { + return false + } + readyBranch = func() { + c.recValue = &v + f(c, more) + } + return true + }, + } + v, ok, more := c.receiveAsyncImpl(callback) + if ok || !more { + // Select() returns in this case/branch. The callback won't be called for this case. However, callback + // will be called for previous cases/branches. We should set readyBranch so that when other case/branch + // become ready they won't consume the value for this Select() call. + readyBranch = func() { + } + // Avoid assigning pointer to nil interface which makes + // c.RecValue != nil and breaks the nil check at the beginning of receiveAsyncImpl + if more { + c.recValue = &v + } else { + pair.receiveFunc = nil + } + f(c, more) + return + } + // callback closure is added to channel's blockedReceives, we need to clean it up to avoid closure leak + cleanups = append(cleanups, func() { + c.removeReceiveCallback(callback) + }) + } else if pair.sendFunc != nil { + f := *pair.sendFunc + c := pair.channel + callback := &sendCallback{ + value: *pair.sendValue, + fn: func() bool { + if readyBranch != nil { + return false + } + readyBranch = func() { + f() + } + return true + }, + } + ok := c.sendAsyncImpl(*pair.sendValue, callback) + if ok { + // Select() returns in this case/branch. The callback won't be called for this case. However, callback + // will be called for previous cases/branches. We should set readyBranch so that when other case/branch + // become ready they won't consume the value for this Select() call. + readyBranch = func() { + } + f() + return + } + // callback closure is added to channel's blockedSends, we need to clean it up to avoid closure leak + cleanups = append(cleanups, func() { + c.removeSendCallback(callback) + }) + } else if pair.futureFunc != nil { + p := pair + f := *p.futureFunc + callback := &receiveCallback{ + fn: func(v interface{}, more bool) bool { + if readyBranch != nil { + return false + } + readyBranch = func() { + p.futureFunc = nil + f(p.future) + } + return true + }, + } + + _, ok, _ := p.future.GetAsync(callback) + if ok { + // Select() returns in this case/branch. The callback won't be called for this case. However, callback + // will be called for previous cases/branches. We should set readyBranch so that when other case/branch + // become ready they won't consume the value for this Select() call. + readyBranch = func() { + } + p.futureFunc = nil + f(p.future) + return + } + // callback closure is added to future's channel's blockedReceives, need to clean up to avoid leak + cleanups = append(cleanups, func() { + p.future.RemoveReceiveCallback(callback) + }) + } + } + if s.defaultFunc != nil { + f := *s.defaultFunc + f() + return + } + for { + if readyBranch != nil { + readyBranch() + state.unblocked() + return + } + state.yield("blocked on " + s.name + ".Select") + } +} + +// NewWorkflowDefinition creates a WorkflowDefinition from a Workflow +func newSyncWorkflowDefinition(workflow workflow) *syncWorkflowDefinition { + return &syncWorkflowDefinition{workflow: workflow} +} + +func getValidatedWorkflowFunction(workflowFunc interface{}, args []interface{}, dataConverter converter.DataConverter, r *registry) (*WorkflowType, *commonpb.Payloads, error) { + if err := validateFunctionArgs(workflowFunc, args, true); err != nil { + return nil, nil, err + } + + fnName, err := getWorkflowFunctionName(r, workflowFunc) + if err != nil { + return nil, nil, err + } + + if dataConverter == nil { + dataConverter = converter.GetDefaultDataConverter() + } + input, err := encodeArgs(dataConverter, args) + if err != nil { + return nil, nil, err + } + return &WorkflowType{Name: fnName}, input, nil +} + +func getWorkflowEnvOptions(ctx Context) *WorkflowOptions { + options := ctx.Value(workflowEnvOptionsContextKey) + if options != nil { + return options.(*WorkflowOptions) + } + return nil +} + +func setWorkflowEnvOptionsIfNotExist(ctx Context) Context { + options := getWorkflowEnvOptions(ctx) + var newOptions WorkflowOptions + if options != nil { + newOptions = *options + } else { + newOptions.signalChannels = make(map[string]Channel) + newOptions.requestedSignalChannels = make(map[string]*requestedSignalChannel) + newOptions.queryHandlers = make(map[string]*queryHandler) + newOptions.updateHandlers = make(map[string]*updateHandler) + newOptions.runningUpdatesHandles = make(map[string]UpdateInfo) + } + if newOptions.DataConverter == nil { + newOptions.DataConverter = converter.GetDefaultDataConverter() + } + + return WithValue(ctx, workflowEnvOptionsContextKey, &newOptions) +} + +func getDataConverterFromWorkflowContext(ctx Context) converter.DataConverter { + options := getWorkflowEnvOptions(ctx) + var dataConverter converter.DataConverter + + if options != nil && options.DataConverter != nil { + dataConverter = options.DataConverter + } else { + dataConverter = converter.GetDefaultDataConverter() + } + + return WithWorkflowContext(ctx, dataConverter) +} + +func getRegistryFromWorkflowContext(ctx Context) *registry { + env := getWorkflowEnvironment(ctx) + return env.GetRegistry() +} + +// getSignalChannel finds the associated channel for the signal. +func (w *WorkflowOptions) getSignalChannel(ctx Context, signalName string) ReceiveChannel { + if ch, ok := w.signalChannels[signalName]; ok { + return ch + } + ch := NewNamedBufferedChannel(ctx, signalName, defaultSignalChannelSize) + w.signalChannels[signalName] = ch + return ch +} + +// GetUnhandledSignalNames returns signal names that have unconsumed signals. +func GetUnhandledSignalNames(ctx Context) []string { + return getWorkflowEnvOptions(ctx).getUnhandledSignalNames() +} + +// GetCurrentDetails gets the previously-set current details. +// +// NOTE: Experimental +func GetCurrentDetails(ctx Context) string { + return getWorkflowEnvOptions(ctx).currentDetails +} + +// SetCurrentDetails sets the current details. +// +// NOTE: Experimental +func SetCurrentDetails(ctx Context, details string) { + getWorkflowEnvOptions(ctx).currentDetails = details +} + +func getWorkflowMetadata(ctx Context) (*sdk.WorkflowMetadata, error) { + info := GetWorkflowInfo(ctx) + eo := getWorkflowEnvOptions(ctx) + ret := &sdk.WorkflowMetadata{ + Definition: &sdk.WorkflowDefinition{ + Type: info.WorkflowType.Name, + QueryDefinitions: []*sdk.WorkflowInteractionDefinition{ + { + Name: QueryTypeStackTrace, + Description: "Current stack trace", + }, + { + Name: QueryTypeOpenSessions, + Description: "Open sessions on the workflow", + }, + { + Name: QueryTypeWorkflowMetadata, + Description: "Metadata about the workflow", + }, + }, + }, + CurrentDetails: eo.currentDetails, + } + // Queries + for k, v := range eo.queryHandlers { + ret.Definition.QueryDefinitions = append(ret.Definition.QueryDefinitions, &sdk.WorkflowInteractionDefinition{ + Name: k, + Description: v.options.Description, + }) + } + // Signals + for k, v := range eo.requestedSignalChannels { + ret.Definition.SignalDefinitions = append(ret.Definition.SignalDefinitions, &sdk.WorkflowInteractionDefinition{ + Name: k, + Description: v.options.Description, + }) + } + // Updates + for k, v := range eo.updateHandlers { + ret.Definition.UpdateDefinitions = append(ret.Definition.UpdateDefinitions, &sdk.WorkflowInteractionDefinition{ + Name: k, + Description: v.description, + }) + } + // Sort interaction definitions + sortWorkflowInteractionDefinitions(ret.Definition.QueryDefinitions) + sortWorkflowInteractionDefinitions(ret.Definition.SignalDefinitions) + sortWorkflowInteractionDefinitions(ret.Definition.UpdateDefinitions) + return ret, nil +} + +func sortWorkflowInteractionDefinitions(defns []*sdk.WorkflowInteractionDefinition) { + sort.Slice(defns, func(i, j int) bool { return defns[i].Name < defns[j].Name }) +} + +// getUnhandledSignalNames returns signal names that have unconsumed signals. +func (w *WorkflowOptions) getUnhandledSignalNames() []string { + var unhandledSignals []string + for k, c := range w.signalChannels { + ch := c.(*channelImpl) + v, ok, _ := ch.receiveAsyncImpl(nil) + if ok { + unhandledSignals = append(unhandledSignals, k) + ch.recValue = &v + } + } + return unhandledSignals +} + +func (w *WorkflowOptions) getRunningUpdateHandles() map[string]UpdateInfo { + return w.runningUpdatesHandles +} + +func (d *decodeFutureImpl) Get(ctx Context, valuePtr interface{}) error { + more := d.futureImpl.channel.Receive(ctx, nil) + if more { + panic("not closed") + } + if !d.futureImpl.ready { + panic("not ready") + } + if d.futureImpl.err != nil || d.futureImpl.value == nil || valuePtr == nil { + return d.futureImpl.err + } + rf := reflect.ValueOf(valuePtr) + if rf.Type().Kind() != reflect.Ptr { + return errors.New("valuePtr parameter is not a pointer") + } + dataConverter := getDataConverterFromWorkflowContext(ctx) + err := dataConverter.FromPayloads(d.futureImpl.value.(*commonpb.Payloads), valuePtr) + if err != nil { + return err + } + return d.futureImpl.err +} + +// newDecodeFuture creates a new future as well as associated Settable that is used to set its value. +// fn - the decoded value needs to be validated against a function. +func newDecodeFuture(ctx Context, fn interface{}) (Future, Settable) { + impl := &decodeFutureImpl{ + &futureImpl{channel: NewChannel(ctx).(*channelImpl)}, fn} + return impl, impl +} + +// setQueryHandler sets query handler for given queryType. +func setQueryHandler(ctx Context, queryType string, handler interface{}, options QueryHandlerOptions) error { + qh := &queryHandler{ + fn: handler, + queryType: queryType, + dataConverter: getDataConverterFromWorkflowContext(ctx), + options: options, + } + err := validateQueryHandlerFn(qh.fn) + if err != nil { + return err + } + + getWorkflowEnvOptions(ctx).queryHandlers[queryType] = qh + return nil +} + +// setUpdateHandler sets update handler for a given update name. +func setUpdateHandler(ctx Context, updateName string, handler interface{}, opts UpdateHandlerOptions) error { + uh, err := newUpdateHandler(updateName, handler, opts) + if err != nil { + return err + } + getWorkflowEnvOptions(ctx).updateHandlers[updateName] = uh + if getWorkflowEnvironment(ctx).TryUse(SDKPriorityUpdateHandling) { + getWorkflowEnvironment(ctx).HandleQueuedUpdates(updateName) + state := getState(ctx) + defer state.unblocked() + state.yield("letting any updates waiting on a handler run") + } + return nil +} + +// validateEquivalentParams verifies that both arguments are functions and that +// said functions take the exact same parameter types in the same order but not +// considering the presence or absence of a workflow.Context parameter in the +// zeroth position. +func validateEquivalentParams(fn1, fn2 interface{}) error { + fn1Type := reflect.TypeOf(fn1) + fn2Type := reflect.TypeOf(fn2) + + if fn1Type.Kind() != reflect.Func { + return fmt.Errorf("type must be function but was %s", fn1Type.Kind()) + } + + if fn2Type.Kind() != reflect.Func { + return fmt.Errorf("type must be function but was %s", fn1Type.Kind()) + } + + ctxType := reflect.TypeOf(new(Context)).Elem() + extractRelevantParamTypes := func(t reflect.Type) []reflect.Type { + out := make([]reflect.Type, 0, t.NumIn()) + for i := 0; i < t.NumIn(); i++ { + paramType := t.In(i) + if i == 0 && paramType.Implements(ctxType) { + // ignore the presence of a workflow.Context as a first param + continue + } + out = append(out, paramType) + } + return out + } + + fn1ParamTypes := extractRelevantParamTypes(fn1Type) + fn2ParamTypes := extractRelevantParamTypes(fn2Type) + + if len(fn1ParamTypes) != len(fn2ParamTypes) { + return errors.New("functions have different numbers of parameters") + } + + for i := 0; i < len(fn1ParamTypes); i++ { + fn1ParamType := fn1ParamTypes[i] + fn2ParamType := fn2ParamTypes[i] + if fn1ParamType != fn2ParamType { + return fmt.Errorf("functions differ at parameter %v; %v != %v", i, fn1ParamType, fn2ParamType) + } + } + return nil +} + +func validateQueryHandlerFn(fn interface{}) error { + fnType := reflect.TypeOf(fn) + if fnType.Kind() != reflect.Func { + return fmt.Errorf("handler must be function but was %s", fnType.Kind()) + } + + if fnType.NumOut() != 2 { + return fmt.Errorf( + "handler must return 2 values (serializable result and error), but found %d return values", fnType.NumOut(), + ) + } + + if !isValidResultType(fnType.Out(0)) { + return fmt.Errorf( + "first return value of handler must be serializable but found: %v", fnType.Out(0).Kind(), + ) + } + if !isError(fnType.Out(1)) { + return fmt.Errorf( + "second return value of handler must be error but found %v", fnType.Out(fnType.NumOut()-1).Kind(), + ) + } + return nil +} + +func (h *queryHandler) execute(input []interface{}) (result interface{}, err error) { + // if query handler panic, convert it to error + defer func() { + if p := recover(); p != nil { + result = nil + st := getStackTraceRaw("query handler [panic]:", 7, 0) + if p == panicIllegalAccessCoroutineState { + // query handler code try to access workflow functions outside of workflow context, make error message + // more descriptive and clear. + p = "query handler must not use temporal context to do things like workflow.NewChannel(), " + + "workflow.Go() or to call any workflow blocking functions like Channel.Get() or Future.Get()" + } + err = fmt.Errorf("query handler panic: %v, stack trace: %v", p, st) + } + }() + + return executeFunction(h.fn, input) +} + +// Add adds delta, which may be negative, to the WaitGroup counter. +// If the counter becomes zero, all goroutines blocked on Wait are released. +// If the counter goes negative, Add panics. +// +// Note that calls with a positive delta that occur when the counter is zero +// must happen before a Wait. Calls with a negative delta, or calls with a +// positive delta that start when the counter is greater than zero, may happen +// at any time. +// Typically this means the calls to Add should execute before the statement +// creating the goroutine or other event to be waited for. +// If a WaitGroup is reused to wait for several independent sets of events, +// new Add calls must happen after all previous Wait calls have returned. +// +// param delta int -> the value to increment the WaitGroup counter by +func (wg *waitGroupImpl) Add(delta int) { + wg.n = wg.n + delta + if wg.n < 0 { + panic("negative WaitGroup counter") + } + if (wg.n > 0) || (!wg.waiting) { + return + } + if wg.n == 0 { + wg.settable.Set(false, nil) + } +} + +// Done decrements the WaitGroup counter by 1, indicating +// that a coroutine in the WaitGroup has completed +func (wg *waitGroupImpl) Done() { + wg.Add(-1) +} + +// Wait blocks and waits for specified number of coroutines to +// finish executing and then unblocks once the counter has reached 0. +// +// param ctx Context -> workflow context +func (wg *waitGroupImpl) Wait(ctx Context) { + assertNotInReadOnlyState(ctx) + if wg.n <= 0 { + return + } + if wg.waiting { + panic("WaitGroup is reused before previous Wait has returned") + } + + wg.waiting = true + if err := wg.future.Get(ctx, &wg.waiting); err != nil { + panic(err) + } + wg.future, wg.settable = NewFuture(ctx) +} + +// Spawn starts a new coroutine with Dispatcher.NewCoroutine +func (us updateSchedulerImpl) Spawn(ctx Context, name string, highPriority bool, f func(Context)) Context { + return us.dispatcher.NewCoroutine(ctx, name, highPriority, f) +} + +// Yield calls the yield function on the coroutineState associated with the +// supplied workflow context. +func (us updateSchedulerImpl) Yield(ctx Context, reason string) { + getState(ctx).yield(reason) +} + +func (m *mutexImpl) Lock(ctx Context) error { + err := Await(ctx, func() bool { + return !m.locked + }) + if err != nil { + return err + } + m.locked = true + return nil +} + +func (m *mutexImpl) TryLock(ctx Context) bool { + assertNotInReadOnlyState(ctx) + if m.locked { + return false + } + m.locked = true + return true +} + +func (m *mutexImpl) Unlock() { + if !m.locked { + panic("Mutex.Unlock() was called on an unlocked mutex") + } + m.locked = false +} + +func (m *mutexImpl) IsLocked() bool { + return m.locked +} + +func (s *semaphoreImpl) Acquire(ctx Context, n int64) error { + err := Await(ctx, func() bool { + return s.size-s.cur >= n + }) + if err != nil { + return err + } + s.cur += n + return nil +} + +func (s *semaphoreImpl) TryAcquire(ctx Context, n int64) bool { + assertNotInReadOnlyState(ctx) + success := s.size-s.cur >= n + if success { + s.cur += n + } + return success +} + +func (s *semaphoreImpl) Release(n int64) { + s.cur -= n + if s.cur < 0 { + panic("Semaphore.Release() released more than held") + } +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_workflow_client.go b/vendor/go.temporal.io/sdk/internal/internal_workflow_client.go new file mode 100644 index 00000000000..31d98ad7b64 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_workflow_client.go @@ -0,0 +1,2274 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "errors" + "fmt" + "io" + "math" + "reflect" + "sync" + "sync/atomic" + "time" + + "github.com/pborman/uuid" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + healthpb "google.golang.org/grpc/health/grpc_health_v1" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/types/known/durationpb" + + "go.temporal.io/api/cloud/cloudservice/v1" + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + historypb "go.temporal.io/api/history/v1" + "go.temporal.io/api/operatorservice/v1" + querypb "go.temporal.io/api/query/v1" + "go.temporal.io/api/sdk/v1" + "go.temporal.io/api/serviceerror" + taskqueuepb "go.temporal.io/api/taskqueue/v1" + updatepb "go.temporal.io/api/update/v1" + "go.temporal.io/api/workflowservice/v1" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/internal/common/retry" + "go.temporal.io/sdk/internal/common/serializer" + "go.temporal.io/sdk/internal/common/util" + "go.temporal.io/sdk/log" +) + +// Assert that structs do indeed implement the interfaces +var ( + _ Client = (*WorkflowClient)(nil) + _ NamespaceClient = (*namespaceClient)(nil) +) + +var ( + errUnsupportedOperation = fmt.Errorf("unsupported operation") + errInvalidServerResponse = fmt.Errorf("invalid server response") + errInvalidWorkflowOperation = fmt.Errorf("invalid WithStartOperation") +) + +const ( + defaultGetHistoryTimeout = 65 * time.Second + defaultGetSystemInfoTimeout = 5 * time.Second + pollUpdateTimeout = 60 * time.Second + maxListArchivedWorkflowTimeout = 3 * time.Minute +) + +type ( + // WorkflowClient is the client for starting a workflow execution. + WorkflowClient struct { + workflowService workflowservice.WorkflowServiceClient + conn *grpc.ClientConn + namespace string + registry *registry + logger log.Logger + metricsHandler metrics.Handler + identity string + dataConverter converter.DataConverter + failureConverter converter.FailureConverter + contextPropagators []ContextPropagator + workerInterceptors []WorkerInterceptor + interceptor ClientOutboundInterceptor + excludeInternalFromRetry *atomic.Bool + capabilities *workflowservice.GetSystemInfoResponse_Capabilities + capabilitiesLock sync.RWMutex + eagerDispatcher *eagerWorkflowDispatcher + + // The pointer value is shared across multiple clients. If non-nil, only + // access/mutate atomically. + unclosedClients *int32 + } + + // cloudOperationsClient is the client for managing cloud. + cloudOperationsClient struct { + conn *grpc.ClientConn + logger log.Logger + cloudServiceClient cloudservice.CloudServiceClient + } + + // namespaceClient is the client for managing namespaces. + namespaceClient struct { + workflowService workflowservice.WorkflowServiceClient + connectionCloser io.Closer + metricsHandler metrics.Handler + logger log.Logger + identity string + } + + // WorkflowRun represents a started non child workflow + WorkflowRun interface { + // GetID return workflow ID, which will be same as StartWorkflowOptions.ID if provided. + GetID() string + + // GetRunID return the first started workflow run ID (please see below) - + // empty string if no such run. Note, this value may change after Get is + // called if there was a later run for this run. + GetRunID() string + + // Get will fill the workflow execution result to valuePtr, if workflow + // execution is a success, or return corresponding error. If valuePtr is + // nil, valuePtr will be ignored and only the corresponding error of the + // workflow will be returned (nil on workflow execution success). + // This is a blocking API. + // + // This call will follow execution runs to the latest result for this run + // instead of strictly returning this run's result. This means that if the + // workflow returned ContinueAsNewError, has a more recent cron execution, + // or has a new run ID on failure (i.e. a retry), this will wait and return + // the result for the latest run in the chain. To strictly get the result + // for this run without following to the latest, use GetWithOptions and set + // the DisableFollowingRuns option to true. + // + // Note, values should not be reused for extraction here because merging on + // top of existing values may result in unexpected behavior similar to + // json.Unmarshal. + Get(ctx context.Context, valuePtr interface{}) error + + // GetWithOptions will fill the workflow execution result to valuePtr, if + // workflow execution is a success, or return corresponding error. If + // valuePtr is nil, valuePtr will be ignored and only the corresponding + // error of the workflow will be returned (nil on workflow execution success). + // This is a blocking API. + // + // Note, values should not be reused for extraction here because merging on + // top of existing values may result in unexpected behavior similar to + // json.Unmarshal. + GetWithOptions(ctx context.Context, valuePtr interface{}, options WorkflowRunGetOptions) error + } + + // WorkflowRunGetOptions are options for WorkflowRun.GetWithOptions. + WorkflowRunGetOptions struct { + // DisableFollowingRuns, if true, will not follow execution chains to the + // latest run. By default when this is false, getting the result of a + // workflow may not use the literal run ID but instead follow to later runs + // if the workflow returned a ContinueAsNewError, has a later cron, or is + // retried on failure. + DisableFollowingRuns bool + } + + // workflowRunImpl is an implementation of WorkflowRun + workflowRunImpl struct { + workflowType string + workflowID string + firstRunID string + currentRunID *util.OnceCell + iterFn func(ctx context.Context, runID string) HistoryEventIterator + dataConverter converter.DataConverter + failureConverter converter.FailureConverter + registry *registry + } + + // HistoryEventIterator represents the interface for + // history event iterator + HistoryEventIterator interface { + // HasNext return whether this iterator has next value + HasNext() bool + // Next returns the next history events and error + // The errors it can return: + // - serviceerror.NotFound + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + Next() (*historypb.HistoryEvent, error) + } + + // historyEventIteratorImpl is the implementation of HistoryEventIterator + historyEventIteratorImpl struct { + // whether this iterator is initialized + initialized bool + // local cached history events and corresponding consuming index + nextEventIndex int + events []*historypb.HistoryEvent + // token to get next page of history events + nexttoken []byte + // err when getting next page of history events + err error + // func which use a next token to get next page of history events + paginate func(nexttoken []byte) (*workflowservice.GetWorkflowExecutionHistoryResponse, error) + } + + // QueryRejectedError is a wrapper for QueryRejected + QueryRejectedError struct { + queryRejected *querypb.QueryRejected + } +) + +// ExecuteWorkflow starts a workflow execution and returns a WorkflowRun that will allow you to wait until this workflow +// reaches the end state, such as workflow finished successfully or timeout. +// The user can use this to start using a functor like below and get the workflow execution result, as EncodedValue +// Either by +// +// ExecuteWorkflow(options, "workflowTypeName", arg1, arg2, arg3) +// or +// ExecuteWorkflow(options, workflowExecuteFn, arg1, arg2, arg3) +// +// The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is +// subjected to change in the future. +// NOTE: the context.Context should have a fairly large timeout, since workflow execution may take a while to be finished +func (wc *WorkflowClient) ExecuteWorkflow(ctx context.Context, options StartWorkflowOptions, workflow interface{}, args ...interface{}) (WorkflowRun, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + // Default workflow ID + if options.ID == "" { + options.ID = uuid.New() + } + + // Validate function and get name + if err := validateFunctionArgs(workflow, args, true); err != nil { + return nil, err + } + workflowType, err := getWorkflowFunctionName(wc.registry, workflow) + if err != nil { + return nil, err + } + + // Set header before interceptor run + ctx = contextWithNewHeader(ctx) + + // Run via interceptor + return wc.interceptor.ExecuteWorkflow(ctx, &ClientExecuteWorkflowInput{ + Options: &options, + WorkflowType: workflowType, + Args: args, + }) +} + +// GetWorkflow gets a workflow execution and returns a WorkflowRun that will allow you to wait until this workflow +// reaches the end state, such as workflow finished successfully or timeout. +// The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is +// subjected to change in the future. +func (wc *WorkflowClient) GetWorkflow(ctx context.Context, workflowID string, runID string) WorkflowRun { + // We intentionally don't "ensureIntialized" here because there is no direct + // error return path. Rather we let GetWorkflowHistory do it. + + iterFn := func(fnCtx context.Context, fnRunID string) HistoryEventIterator { + return wc.GetWorkflowHistory(fnCtx, workflowID, fnRunID, true, enumspb.HISTORY_EVENT_FILTER_TYPE_CLOSE_EVENT) + } + + // The ID may not actually have been set - if not, we have to (lazily) ask the server for info about the workflow + // execution and extract run id from there. This is definitely less efficient than it could be if there was a more + // specific rpc method for this, or if there were more granular history filters - in which case it could be + // extracted from the `iterFn` inside of `workflowRunImpl` + var runIDCell util.OnceCell + if runID == "" { + fetcher := func() string { + execData, _ := wc.DescribeWorkflowExecution(ctx, workflowID, runID) + wei := execData.GetWorkflowExecutionInfo() + if wei != nil { + execution := wei.GetExecution() + if execution != nil { + return execution.RunId + } + } + return "" + } + runIDCell = util.LazyOnceCell(fetcher) + } else { + runIDCell = util.PopulatedOnceCell(runID) + } + + return &workflowRunImpl{ + workflowID: workflowID, + firstRunID: runID, + currentRunID: &runIDCell, + iterFn: iterFn, + dataConverter: wc.dataConverter, + failureConverter: wc.failureConverter, + registry: wc.registry, + } +} + +// SignalWorkflow signals a workflow in execution. +func (wc *WorkflowClient) SignalWorkflow(ctx context.Context, workflowID string, runID string, signalName string, arg interface{}) error { + if err := wc.ensureInitialized(ctx); err != nil { + return err + } + + // Set header before interceptor run + ctx = contextWithNewHeader(ctx) + + return wc.interceptor.SignalWorkflow(ctx, &ClientSignalWorkflowInput{ + WorkflowID: workflowID, + RunID: runID, + SignalName: signalName, + Arg: arg, + }) +} + +// SignalWithStartWorkflow sends a signal to a running workflow. +// If the workflow is not running or not found, it starts the workflow and then sends the signal in transaction. +func (wc *WorkflowClient) SignalWithStartWorkflow(ctx context.Context, workflowID string, signalName string, signalArg interface{}, + options StartWorkflowOptions, workflowFunc interface{}, workflowArgs ...interface{}, +) (WorkflowRun, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + // Due to the ambiguous way to provide workflow IDs, if options contains an + // ID, it must match the parameter + if options.ID != "" && options.ID != workflowID { + return nil, fmt.Errorf("workflow ID from options not used, must be unset or match workflow ID parameter") + } + if options.WithStartOperation != nil { + return nil, fmt.Errorf("option WithStartOperation is not allowed") + } + + // Default workflow ID to UUID + options.ID = workflowID + if options.ID == "" { + options.ID = uuid.New() + } + + // Validate function and get name + if err := validateFunctionArgs(workflowFunc, workflowArgs, true); err != nil { + return nil, err + } + workflowType, err := getWorkflowFunctionName(wc.registry, workflowFunc) + if err != nil { + return nil, err + } + + // Set header before interceptor run + ctx = contextWithNewHeader(ctx) + + // Run via interceptor + return wc.interceptor.SignalWithStartWorkflow(ctx, &ClientSignalWithStartWorkflowInput{ + SignalName: signalName, + SignalArg: signalArg, + Options: &options, + WorkflowType: workflowType, + Args: workflowArgs, + }) +} + +// CancelWorkflow cancels a workflow in execution. It allows workflow to properly clean up and gracefully close. +// workflowID is required, other parameters are optional. +// If runID is omit, it will terminate currently running workflow (if there is one) based on the workflowID. +func (wc *WorkflowClient) CancelWorkflow(ctx context.Context, workflowID string, runID string) error { + if err := wc.ensureInitialized(ctx); err != nil { + return err + } + + return wc.interceptor.CancelWorkflow(ctx, &ClientCancelWorkflowInput{WorkflowID: workflowID, RunID: runID}) +} + +// TerminateWorkflow terminates a workflow execution. +// workflowID is required, other parameters are optional. +// If runID is omit, it will terminate currently running workflow (if there is one) based on the workflowID. +func (wc *WorkflowClient) TerminateWorkflow(ctx context.Context, workflowID string, runID string, reason string, details ...interface{}) error { + if err := wc.ensureInitialized(ctx); err != nil { + return err + } + + return wc.interceptor.TerminateWorkflow(ctx, &ClientTerminateWorkflowInput{ + WorkflowID: workflowID, + RunID: runID, + Reason: reason, + Details: details, + }) +} + +// GetWorkflowHistory return a channel which contains the history events of a given workflow +func (wc *WorkflowClient) GetWorkflowHistory( + ctx context.Context, + workflowID string, + runID string, + isLongPoll bool, + filterType enumspb.HistoryEventFilterType, +) HistoryEventIterator { + return wc.getWorkflowHistory(ctx, workflowID, runID, isLongPoll, filterType, wc.metricsHandler) +} + +func (wc *WorkflowClient) getWorkflowHistory( + ctx context.Context, + workflowID string, + runID string, + isLongPoll bool, + filterType enumspb.HistoryEventFilterType, + rpcMetricsHandler metrics.Handler, +) HistoryEventIterator { + namespace := wc.namespace + paginate := func(nextToken []byte) (*workflowservice.GetWorkflowExecutionHistoryResponse, error) { + request := &workflowservice.GetWorkflowExecutionHistoryRequest{ + Namespace: namespace, + Execution: &commonpb.WorkflowExecution{ + WorkflowId: workflowID, + RunId: runID, + }, + WaitNewEvent: isLongPoll, + HistoryEventFilterType: filterType, + NextPageToken: nextToken, + SkipArchival: isLongPoll, + } + + var response *workflowservice.GetWorkflowExecutionHistoryResponse + var err error + Loop: + for { + response, err = wc.getWorkflowExecutionHistory(ctx, rpcMetricsHandler, isLongPoll, request, filterType) + if err != nil { + return nil, err + } + if isLongPoll && len(response.History.Events) == 0 && len(response.NextPageToken) != 0 { + request.NextPageToken = response.NextPageToken + continue Loop + } + break Loop + } + return response, nil + } + + return &historyEventIteratorImpl{ + paginate: paginate, + } +} + +func (wc *WorkflowClient) getWorkflowExecutionHistory(ctx context.Context, rpcMetricsHandler metrics.Handler, isLongPoll bool, + request *workflowservice.GetWorkflowExecutionHistoryRequest, filterType enumspb.HistoryEventFilterType, +) (*workflowservice.GetWorkflowExecutionHistoryResponse, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler(rpcMetricsHandler), grpcLongPoll(isLongPoll), defaultGrpcRetryParameters(ctx), func(builder *grpcContextBuilder) { + if isLongPoll { + builder.Timeout = defaultGetHistoryTimeout + } + }) + + defer cancel() + response, err := wc.workflowService.GetWorkflowExecutionHistory(grpcCtx, request) + if err != nil { + return nil, err + } + + if response.RawHistory != nil { + history, err := serializer.DeserializeBlobDataToHistoryEvents(response.RawHistory, filterType) + if err != nil { + return nil, err + } + response.History = history + } + return response, err +} + +// CompleteActivity reports activity completed. activity Execute method can return activity.ErrResultPending to +// indicate the activity is not completed when it's Execute method returns. In that case, this CompleteActivity() method +// should be called when that activity is completed with the actual result and error. If err is nil, activity task +// completed event will be reported; if err is CanceledError, activity task canceled event will be reported; otherwise, +// activity task failed event will be reported. +func (wc *WorkflowClient) CompleteActivity(ctx context.Context, taskToken []byte, result interface{}, err error) error { + if err := wc.ensureInitialized(ctx); err != nil { + return err + } + + if taskToken == nil { + return errors.New("invalid task token provided") + } + + dataConverter := WithContext(ctx, wc.dataConverter) + var data *commonpb.Payloads + if result != nil { + var err0 error + data, err0 = encodeArg(dataConverter, result) + if err0 != nil { + return err0 + } + } + + // We do allow canceled error to be passed here + cancelAllowed := true + request := convertActivityResultToRespondRequest(wc.identity, taskToken, + data, err, wc.dataConverter, wc.failureConverter, wc.namespace, cancelAllowed, nil) + return reportActivityComplete(ctx, wc.workflowService, request, wc.metricsHandler) +} + +// CompleteActivityByID reports activity completed. Similar to CompleteActivity +// It takes namespace name, workflowID, runID, activityID as arguments. +func (wc *WorkflowClient) CompleteActivityByID(ctx context.Context, namespace, workflowID, runID, activityID string, + result interface{}, err error, +) error { + if activityID == "" || workflowID == "" || namespace == "" { + return errors.New("empty activity or workflow id or namespace") + } + + dataConverter := WithContext(ctx, wc.dataConverter) + var data *commonpb.Payloads + if result != nil { + var err0 error + data, err0 = encodeArg(dataConverter, result) + if err0 != nil { + return err0 + } + } + + // We do allow canceled error to be passed here + cancelAllowed := true + request := convertActivityResultToRespondRequestByID(wc.identity, namespace, workflowID, runID, activityID, + data, err, wc.dataConverter, wc.failureConverter, cancelAllowed) + return reportActivityCompleteByID(ctx, wc.workflowService, request, wc.metricsHandler) +} + +// RecordActivityHeartbeat records heartbeat for an activity. +func (wc *WorkflowClient) RecordActivityHeartbeat(ctx context.Context, taskToken []byte, details ...interface{}) error { + if err := wc.ensureInitialized(ctx); err != nil { + return err + } + + dataConverter := WithContext(ctx, wc.dataConverter) + data, err := encodeArgs(dataConverter, details) + if err != nil { + return err + } + return recordActivityHeartbeat(ctx, wc.workflowService, wc.metricsHandler, wc.identity, taskToken, data) +} + +// RecordActivityHeartbeatByID records heartbeat for an activity. +func (wc *WorkflowClient) RecordActivityHeartbeatByID(ctx context.Context, + namespace, workflowID, runID, activityID string, details ...interface{}, +) error { + if err := wc.ensureInitialized(ctx); err != nil { + return err + } + + dataConverter := WithContext(ctx, wc.dataConverter) + data, err := encodeArgs(dataConverter, details) + if err != nil { + return err + } + return recordActivityHeartbeatByID(ctx, wc.workflowService, wc.metricsHandler, wc.identity, namespace, workflowID, runID, activityID, data) +} + +// ListClosedWorkflow gets closed workflow executions based on request filters +// The errors it can throw: +// - serviceerror.InvalidArgument +// - serviceerror.Internal +// - serviceerror.Unavailable +// - serviceerror.NamespaceNotFound +func (wc *WorkflowClient) ListClosedWorkflow(ctx context.Context, request *workflowservice.ListClosedWorkflowExecutionsRequest) (*workflowservice.ListClosedWorkflowExecutionsResponse, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + if request.GetNamespace() == "" { + request.Namespace = wc.namespace + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + response, err := wc.workflowService.ListClosedWorkflowExecutions(grpcCtx, request) + if err != nil { + return nil, err + } + return response, nil +} + +// ListOpenWorkflow gets open workflow executions based on request filters +// The errors it can throw: +// - serviceerror.InvalidArgument +// - serviceerror.Internal +// - serviceerror.Unavailable +// - serviceerror.NamespaceNotFound +func (wc *WorkflowClient) ListOpenWorkflow(ctx context.Context, request *workflowservice.ListOpenWorkflowExecutionsRequest) (*workflowservice.ListOpenWorkflowExecutionsResponse, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + if request.GetNamespace() == "" { + request.Namespace = wc.namespace + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + response, err := wc.workflowService.ListOpenWorkflowExecutions(grpcCtx, request) + if err != nil { + return nil, err + } + return response, nil +} + +// ListWorkflow implementation +func (wc *WorkflowClient) ListWorkflow(ctx context.Context, request *workflowservice.ListWorkflowExecutionsRequest) (*workflowservice.ListWorkflowExecutionsResponse, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + if request.GetNamespace() == "" { + request.Namespace = wc.namespace + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + response, err := wc.workflowService.ListWorkflowExecutions(grpcCtx, request) + if err != nil { + return nil, err + } + return response, nil +} + +// ListArchivedWorkflow implementation +func (wc *WorkflowClient) ListArchivedWorkflow(ctx context.Context, request *workflowservice.ListArchivedWorkflowExecutionsRequest) (*workflowservice.ListArchivedWorkflowExecutionsResponse, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + if request.GetNamespace() == "" { + request.Namespace = wc.namespace + } + timeout := maxListArchivedWorkflowTimeout + now := time.Now() + if ctx != nil { + if expiration, ok := ctx.Deadline(); ok && expiration.After(now) { + timeout = expiration.Sub(now) + if timeout > maxListArchivedWorkflowTimeout { + timeout = maxListArchivedWorkflowTimeout + } else if timeout < minRPCTimeout { + timeout = minRPCTimeout + } + } + } + grpcCtx, cancel := newGRPCContext(ctx, grpcTimeout(timeout), defaultGrpcRetryParameters(ctx)) + defer cancel() + response, err := wc.workflowService.ListArchivedWorkflowExecutions(grpcCtx, request) + if err != nil { + return nil, err + } + return response, nil +} + +// ScanWorkflow implementation +func (wc *WorkflowClient) ScanWorkflow(ctx context.Context, request *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + if request.GetNamespace() == "" { + request.Namespace = wc.namespace + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + response, err := wc.workflowService.ScanWorkflowExecutions(grpcCtx, request) + if err != nil { + return nil, err + } + return response, nil +} + +// CountWorkflow implementation +func (wc *WorkflowClient) CountWorkflow(ctx context.Context, request *workflowservice.CountWorkflowExecutionsRequest) (*workflowservice.CountWorkflowExecutionsResponse, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + if request.GetNamespace() == "" { + request.Namespace = wc.namespace + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + response, err := wc.workflowService.CountWorkflowExecutions(grpcCtx, request) + if err != nil { + return nil, err + } + return response, nil +} + +// GetSearchAttributes implementation +func (wc *WorkflowClient) GetSearchAttributes(ctx context.Context) (*workflowservice.GetSearchAttributesResponse, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + response, err := wc.workflowService.GetSearchAttributes(grpcCtx, &workflowservice.GetSearchAttributesRequest{}) + if err != nil { + return nil, err + } + return response, nil +} + +// DescribeWorkflowExecution returns information about the specified workflow execution. +// The errors it can return: +// - serviceerror.InvalidArgument +// - serviceerror.Internal +// - serviceerror.Unavailable +// - serviceerror.NotFound +func (wc *WorkflowClient) DescribeWorkflowExecution(ctx context.Context, workflowID, runID string) (*workflowservice.DescribeWorkflowExecutionResponse, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + request := &workflowservice.DescribeWorkflowExecutionRequest{ + Namespace: wc.namespace, + Execution: &commonpb.WorkflowExecution{ + WorkflowId: workflowID, + RunId: runID, + }, + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + response, err := wc.workflowService.DescribeWorkflowExecution(grpcCtx, request) + if err != nil { + return nil, err + } + return response, nil +} + +// QueryWorkflow queries a given workflow execution +// workflowID and queryType are required, other parameters are optional. +// - workflow ID of the workflow. +// - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. +// - taskQueue can be default(empty string). If empty string then it will pick the taskQueue of the running execution of that workflow ID. +// - queryType is the type of the query. +// - args... are the optional query parameters. +// The errors it can return: +// - serviceerror.InvalidArgument +// - serviceerror.Internal +// - serviceerror.Unavailable +// - serviceerror.NotFound +// - serviceerror.QueryFailed +func (wc *WorkflowClient) QueryWorkflow(ctx context.Context, workflowID string, runID string, queryType string, args ...interface{}) (converter.EncodedValue, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + // Set header before interceptor run + ctx = contextWithNewHeader(ctx) + + return wc.interceptor.QueryWorkflow(ctx, &ClientQueryWorkflowInput{ + WorkflowID: workflowID, + RunID: runID, + QueryType: queryType, + Args: args, + }) +} + +// UpdateWorkflowOptions is the request to UpdateWorkflow +type UpdateWorkflowOptions struct { + // UpdateID is an application-layer identifier for the requested update. It + // must be unique within the scope of a Namespace+WorkflowID+RunID. + UpdateID string + + // WorkflowID is a required field indicating the workflow which should be + // updated. However, it is optional when using UpdateWithStartWorkflowOperation. + WorkflowID string + + // RunID is an optional field used to identify a specific run of the target + // workflow. If RunID is not provided the latest run will be used. + // Note that it is incompatible with UpdateWithStartWorkflowOperation. + RunID string + + // UpdateName is a required field which specifies the update you want to run. + // See comments at workflow.SetUpdateHandler(ctx Context, updateName string, handler interface{}, opts UpdateHandlerOptions) + // for more details on how to setup update handlers within the target workflow. + UpdateName string + + // Args is an optional field used to identify the arguments passed to the + // update. + Args []interface{} + + // WaitForStage is a required field which specifies which stage to wait until returning. + // See https://docs.temporal.io/workflows#update for more details. + // NOTE: Specifying WorkflowUpdateStageAdmitted is not supported. + WaitForStage WorkflowUpdateStage + + // FirstExecutionRunID specifies the RunID expected to identify the first + // run in the workflow execution chain. If this expectation does not match + // then the server will reject the update request with an error. + // Note that it is incompatible with UpdateWithStartWorkflowOperation. + FirstExecutionRunID string +} + +// WorkflowUpdateHandle is a handle to a workflow execution update process. The +// update may or may not have completed so an instance of this type functions +// similar to a Future with respect to the outcome of the update. If the update +// is rejected or returns an error, the Get function on this type will return +// that error through the output valuePtr. +// NOTE: Experimental +type WorkflowUpdateHandle interface { + // WorkflowID observes the update's workflow ID. + WorkflowID() string + + // RunID observes the update's run ID. + RunID() string + + // UpdateID observes the update's ID. + UpdateID() string + + // Get blocks on the outcome of the update. + Get(ctx context.Context, valuePtr interface{}) error +} + +// GetWorkflowUpdateHandleOptions encapsulates the parameters needed to unambiguously +// refer to a Workflow Update. +// NOTE: Experimental +type GetWorkflowUpdateHandleOptions struct { + // WorkflowID of the target update + WorkflowID string + + // RunID of the target workflow. If blank, use the most recent run + RunID string + + // UpdateID of the target update + UpdateID string +} + +type baseUpdateHandle struct { + ref *updatepb.UpdateRef +} + +// completedUpdateHandle is an UpdateHandle impelementation for use when the outcome +// of the update is already known and the Get call can return immediately. +type completedUpdateHandle struct { + baseUpdateHandle + value converter.EncodedValue + err error +} + +// lazyUpdateHandle represents and update that is not known to have completed +// yet (i.e. the associated updatepb.Outcome is not known) and thus calling Get +// will poll the server for the outcome. +type lazyUpdateHandle struct { + baseUpdateHandle + client *WorkflowClient +} + +// QueryWorkflowWithOptionsRequest is the request to QueryWorkflowWithOptions +type QueryWorkflowWithOptionsRequest struct { + // WorkflowID is a required field indicating the workflow which should be queried. + WorkflowID string + + // RunID is an optional field used to identify a specific run of the queried workflow. + // If RunID is not provided the latest run will be used. + RunID string + + // QueryType is a required field which specifies the query you want to run. + // By default, temporal supports "__stack_trace" as a standard query type, which will return string value + // representing the call stack of the target workflow. The target workflow could also setup different query handler to handle custom query types. + // See comments at workflow.SetQueryHandler(ctx Context, queryType string, handler interface{}) for more details on how to setup query handler within the target workflow. + QueryType string + + // Args is an optional field used to identify the arguments passed to the query. + Args []interface{} + + // QueryRejectCondition is an optional field used to reject queries based on workflow state. + // QUERY_REJECT_CONDITION_NONE indicates that query should not be rejected. + // QUERY_REJECT_CONDITION_NOT_OPEN indicates that query should be rejected if workflow is not open. + // QUERY_REJECT_CONDITION_NOT_COMPLETED_CLEANLY indicates that query should be rejected if workflow did not complete cleanly (e.g. terminated, canceled timeout etc...). + QueryRejectCondition enumspb.QueryRejectCondition + + // Header is an optional header to include with the query. + Header *commonpb.Header +} + +// QueryWorkflowWithOptionsResponse is the response to QueryWorkflowWithOptions +type QueryWorkflowWithOptionsResponse struct { + // QueryResult contains the result of executing the query. + // This will only be set if the query was completed successfully and not rejected. + QueryResult converter.EncodedValue + + // QueryRejected contains information about the query rejection. + QueryRejected *querypb.QueryRejected +} + +// QueryWorkflowWithOptions queries a given workflow execution and returns the query result synchronously. +// See QueryWorkflowWithOptionsRequest and QueryWorkflowWithOptionsResult for more information. +// The errors it can return: +// - serviceerror.InvalidArgument +// - serviceerror.Internal +// - serviceerror.Unavailable +// - serviceerror.NotFound +// - serviceerror.QueryFailed +func (wc *WorkflowClient) QueryWorkflowWithOptions(ctx context.Context, request *QueryWorkflowWithOptionsRequest) (*QueryWorkflowWithOptionsResponse, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + // Set header before interceptor run + ctx, err := contextWithHeaderPropagated(ctx, request.Header, wc.contextPropagators) + if err != nil { + return nil, err + } + + result, err := wc.interceptor.QueryWorkflow(ctx, &ClientQueryWorkflowInput{ + WorkflowID: request.WorkflowID, + RunID: request.RunID, + QueryType: request.QueryType, + Args: request.Args, + QueryRejectCondition: request.QueryRejectCondition, + }) + if err != nil { + var qerr *QueryRejectedError + if errors.As(err, &qerr) { + return &QueryWorkflowWithOptionsResponse{ + QueryRejected: qerr.QueryRejected(), + }, nil + } + return nil, err + } + return &QueryWorkflowWithOptionsResponse{ + QueryResult: result, + }, nil +} + +// DescribeTaskQueue returns information about the target taskqueue, right now this API returns the +// pollers which polled this taskqueue in last few minutes. +// - taskqueue name of taskqueue +// - taskqueueType type of taskqueue, can be workflow or activity +// The errors it can return: +// - serviceerror.InvalidArgument +// - serviceerror.Internal +// - serviceerror.Unavailable +// - serviceerror.NotFound +func (wc *WorkflowClient) DescribeTaskQueue(ctx context.Context, taskQueue string, taskQueueType enumspb.TaskQueueType) (*workflowservice.DescribeTaskQueueResponse, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + request := &workflowservice.DescribeTaskQueueRequest{ + Namespace: wc.namespace, + TaskQueue: &taskqueuepb.TaskQueue{Name: taskQueue, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, + TaskQueueType: taskQueueType, + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + resp, err := wc.workflowService.DescribeTaskQueue(grpcCtx, request) + if err != nil { + return nil, err + } + + return resp, nil +} + +// ResetWorkflowExecution reset an existing workflow execution to WorkflowTaskFinishEventId(exclusive). +// And it will immediately terminating the current execution instance. +// RequestId is used to deduplicate requests. It will be autogenerated if not set. +func (wc *WorkflowClient) ResetWorkflowExecution(ctx context.Context, request *workflowservice.ResetWorkflowExecutionRequest) (*workflowservice.ResetWorkflowExecutionResponse, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + if request != nil && request.GetRequestId() == "" { + request.RequestId = uuid.New() + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + resp, err := wc.workflowService.ResetWorkflowExecution(grpcCtx, request) + if err != nil { + return nil, err + } + + return resp, nil +} + +// UpdateWorkerBuildIdCompatibility allows you to update the worker-build-id based version sets for a particular +// task queue. This is used in conjunction with workers who specify their build id and thus opt into the +// feature. +func (wc *WorkflowClient) UpdateWorkerBuildIdCompatibility(ctx context.Context, options *UpdateWorkerBuildIdCompatibilityOptions) error { + if err := wc.ensureInitialized(ctx); err != nil { + return err + } + + request, err := options.validateAndConvertToProto() + if err != nil { + return err + } + request.Namespace = wc.namespace + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err = wc.workflowService.UpdateWorkerBuildIdCompatibility(grpcCtx, request) + return err +} + +// GetWorkerBuildIdCompatibility returns the worker-build-id based version sets for a particular task queue. +func (wc *WorkflowClient) GetWorkerBuildIdCompatibility(ctx context.Context, options *GetWorkerBuildIdCompatibilityOptions) (*WorkerBuildIDVersionSets, error) { + if options.MaxSets < 0 { + return nil, errors.New("maxDepth must be >= 0") + } + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + request := &workflowservice.GetWorkerBuildIdCompatibilityRequest{ + Namespace: wc.namespace, + TaskQueue: options.TaskQueue, + MaxSets: int32(options.MaxSets), + } + resp, err := wc.workflowService.GetWorkerBuildIdCompatibility(grpcCtx, request) + if err != nil { + return nil, err + } + converted := workerVersionSetsFromProtoResponse(resp) + return converted, nil +} + +// GetWorkerTaskReachability returns which versions are is still in use by open or closed workflows. +func (wc *WorkflowClient) GetWorkerTaskReachability(ctx context.Context, options *GetWorkerTaskReachabilityOptions) (*WorkerTaskReachability, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + request := &workflowservice.GetWorkerTaskReachabilityRequest{ + Namespace: wc.namespace, + BuildIds: options.BuildIDs, + TaskQueues: options.TaskQueues, + Reachability: taskReachabilityToProto(options.Reachability), + } + resp, err := wc.workflowService.GetWorkerTaskReachability(grpcCtx, request) + if err != nil { + return nil, err + } + converted := workerTaskReachabilityFromProtoResponse(resp) + return converted, nil +} + +// DescribeTaskQueueEnhanced returns information about the target task queue, broken down by Build Id: +// - List of pollers +// - Workflow Reachability status +// - Backlog info for Workflow and/or Activity tasks +// +// WARNING: Worker versioning is currently experimental, and requires server 1.24+ +func (wc *WorkflowClient) DescribeTaskQueueEnhanced(ctx context.Context, options DescribeTaskQueueEnhancedOptions) (TaskQueueDescription, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return TaskQueueDescription{}, err + } + + request, err := options.validateAndConvertToProto(wc.namespace) + if err != nil { + return TaskQueueDescription{}, err + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + resp, err := wc.workflowService.DescribeTaskQueue(grpcCtx, request) + if err != nil { + return TaskQueueDescription{}, err + } + + err = detectTaskQueueEnhancedNotSupported(resp) + if err != nil { + return TaskQueueDescription{}, err + } + + return taskQueueDescriptionFromResponse(resp), nil +} + +// UpdateWorkerVersioningRules allows updating the worker-build-id based assignment and redirect rules for a given +// task queue. This is used in conjunction with workers who specify their build id and thus opt into the feature. +// The errors it can return: +// - serviceerror.FailedPrecondition when the conflict token is invalid +func (wc *WorkflowClient) UpdateWorkerVersioningRules(ctx context.Context, options UpdateWorkerVersioningRulesOptions) (*WorkerVersioningRules, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + request, err := options.validateAndConvertToProto(wc.namespace) + if err != nil { + return nil, err + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + resp, err := wc.workflowService.UpdateWorkerVersioningRules(grpcCtx, request) + if err != nil { + return nil, err + } + return workerVersioningRulesFromProtoUpdateResponse(resp), nil +} + +// GetWorkerVersioningRules returns the worker-build-id assignment and redirect rules for a task queue. +func (wc *WorkflowClient) GetWorkerVersioningRules(ctx context.Context, options GetWorkerVersioningOptions) (*WorkerVersioningRules, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + request, err := options.validateAndConvertToProto(wc.namespace) + if err != nil { + return nil, err + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + resp, err := wc.workflowService.GetWorkerVersioningRules(grpcCtx, request) + if err != nil { + return nil, err + } + return workerVersioningRulesFromProtoGetResponse(resp), nil +} + +func (wc *WorkflowClient) GetWorkflowUpdateHandle(ref GetWorkflowUpdateHandleOptions) WorkflowUpdateHandle { + return &lazyUpdateHandle{ + client: wc, + baseUpdateHandle: baseUpdateHandle{ + ref: &updatepb.UpdateRef{ + WorkflowExecution: &commonpb.WorkflowExecution{ + WorkflowId: ref.WorkflowID, + RunId: ref.RunID, + }, + UpdateId: ref.UpdateID, + }, + }, + } +} + +// PollWorkflowUpdate sends a request for the outcome of the specified update +// through the interceptor chain. +func (wc *WorkflowClient) PollWorkflowUpdate( + ctx context.Context, + ref *updatepb.UpdateRef, +) (*ClientPollWorkflowUpdateOutput, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + ctx = contextWithNewHeader(ctx) + return wc.interceptor.PollWorkflowUpdate(ctx, &ClientPollWorkflowUpdateInput{ + UpdateRef: ref, + }) +} + +func (wc *WorkflowClient) UpdateWorkflow( + ctx context.Context, + options UpdateWorkflowOptions, +) (WorkflowUpdateHandle, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + in, err := createUpdateWorkflowInput(options) + if err != nil { + return nil, err + } + + ctx = contextWithNewHeader(ctx) + + return wc.interceptor.UpdateWorkflow(ctx, in) +} + +// CheckHealthRequest is a request for Client.CheckHealth. +type CheckHealthRequest struct{} + +// CheckHealthResponse is a response for Client.CheckHealth. +type CheckHealthResponse struct{} + +// CheckHealth performs a server health check using the gRPC health check +// API. If the check fails, an error is returned. +func (wc *WorkflowClient) CheckHealth(ctx context.Context, request *CheckHealthRequest) (*CheckHealthResponse, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + // Ignore request/response for now, they are empty + resp, err := healthpb.NewHealthClient(wc.conn).Check(ctx, &healthpb.HealthCheckRequest{ + Service: "temporal.api.workflowservice.v1.WorkflowService", + }) + if err != nil { + return nil, fmt.Errorf("health check error: %w", err) + } else if resp.Status != healthpb.HealthCheckResponse_SERVING { + return nil, fmt.Errorf("health check returned unhealthy status: %v", resp.Status) + } + return &CheckHealthResponse{}, nil +} + +// WorkflowService implements Client.WorkflowService. +func (wc *WorkflowClient) WorkflowService() workflowservice.WorkflowServiceClient { + return wc.workflowService +} + +// OperatorService implements Client.OperatorService. +func (wc *WorkflowClient) OperatorService() operatorservice.OperatorServiceClient { + return operatorservice.NewOperatorServiceClient(wc.conn) +} + +// Get capabilities, lazily fetching from server if not already obtained. +func (wc *WorkflowClient) loadCapabilities(ctx context.Context, getSystemInfoTimeout time.Duration) (*workflowservice.GetSystemInfoResponse_Capabilities, error) { + // While we want to memoize the result here, we take care not to lock during + // the call. This means that in racy situations where this is called multiple + // times at once, it may result in multiple calls. This is far more preferable + // than locking on the call itself. + wc.capabilitiesLock.RLock() + capabilities := wc.capabilities + wc.capabilitiesLock.RUnlock() + if capabilities != nil { + return capabilities, nil + } + + // Fetch the capabilities + if getSystemInfoTimeout == 0 { + getSystemInfoTimeout = defaultGetSystemInfoTimeout + } + grpcCtx, cancel := newGRPCContext(ctx, grpcTimeout(getSystemInfoTimeout)) + defer cancel() + resp, err := wc.workflowService.GetSystemInfo(grpcCtx, &workflowservice.GetSystemInfoRequest{}) + // We ignore unimplemented + if _, isUnimplemented := err.(*serviceerror.Unimplemented); err != nil && !isUnimplemented { + return nil, fmt.Errorf("failed reaching server: %w", err) + } + if resp != nil && resp.Capabilities != nil { + capabilities = resp.Capabilities + } else { + capabilities = &workflowservice.GetSystemInfoResponse_Capabilities{} + } + + // Store and return. We intentionally don't check if we're overwriting as we + // accept last-success-wins. + wc.capabilitiesLock.Lock() + wc.capabilities = capabilities + // Also set whether we exclude internal from retry + wc.excludeInternalFromRetry.Store(capabilities.InternalErrorDifferentiation) + wc.capabilitiesLock.Unlock() + return capabilities, nil +} + +func (wc *WorkflowClient) ensureInitialized(ctx context.Context) error { + // Just loading the capabilities is enough + _, err := wc.loadCapabilities(ctx, defaultGetSystemInfoTimeout) + return err +} + +// ScheduleClient implements Client.ScheduleClient. +func (wc *WorkflowClient) ScheduleClient() ScheduleClient { + return &scheduleClient{ + workflowClient: wc, + } +} + +// Close client and clean up underlying resources. +func (wc *WorkflowClient) Close() { + // If there's a set of unclosed clients, we have to decrement it and then + // set it to a new pointer of max to prevent decrementing on repeated Close + // calls to this client. If the count has not reached zero, this close call is + // ignored. + if wc.unclosedClients != nil { + remainingUnclosedClients := atomic.AddInt32(wc.unclosedClients, -1) + // Set the unclosed clients to max value so we never try this again + var maxUnclosedClients int32 = math.MaxInt32 + wc.unclosedClients = &maxUnclosedClients + // If there are any remaining, do not close + if remainingUnclosedClients > 0 { + return + } + } + + if wc.conn != nil { + if err := wc.conn.Close(); err != nil { + wc.logger.Warn("unable to close connection", tagError, err) + } + } +} + +func (c *cloudOperationsClient) CloudService() cloudservice.CloudServiceClient { + return c.cloudServiceClient +} + +func (c *cloudOperationsClient) Close() { + if err := c.conn.Close(); err != nil { + c.logger.Warn("unable to close connection", tagError, err) + } +} + +// Register a namespace with temporal server +// The errors it can throw: +// - NamespaceAlreadyExistsError +// - serviceerror.InvalidArgument +// - serviceerror.Internal +// - serviceerror.Unavailable +func (nc *namespaceClient) Register(ctx context.Context, request *workflowservice.RegisterNamespaceRequest) error { + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + var err error + _, err = nc.workflowService.RegisterNamespace(grpcCtx, request) + return err +} + +// Describe a namespace. The namespace has 3 part of information +// NamespaceInfo - Which has Name, Status, Description, Owner Email +// NamespaceConfiguration - Configuration like Workflow Execution Retention Period In Days, Whether to emit metrics. +// ReplicationConfiguration - replication config like clusters and active cluster name +// The errors it can throw: +// - serviceerror.NamespaceNotFound +// - serviceerror.InvalidArgument +// - serviceerror.Internal +// - serviceerror.Unavailable +func (nc *namespaceClient) Describe(ctx context.Context, namespace string) (*workflowservice.DescribeNamespaceResponse, error) { + request := &workflowservice.DescribeNamespaceRequest{ + Namespace: namespace, + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + response, err := nc.workflowService.DescribeNamespace(grpcCtx, request) + if err != nil { + return nil, err + } + return response, nil +} + +// Update a namespace. +// The errors it can throw: +// - serviceerror.NamespaceNotFound +// - serviceerror.InvalidArgument +// - serviceerror.Internal +// - serviceerror.Unavailable +func (nc *namespaceClient) Update(ctx context.Context, request *workflowservice.UpdateNamespaceRequest) error { + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := nc.workflowService.UpdateNamespace(grpcCtx, request) + return err +} + +// Close client and clean up underlying resources. +func (nc *namespaceClient) Close() { + if nc.connectionCloser == nil { + return + } + if err := nc.connectionCloser.Close(); err != nil { + nc.logger.Warn("unable to close connection", tagError, err) + } +} + +func (iter *historyEventIteratorImpl) HasNext() bool { + if iter.nextEventIndex < len(iter.events) || iter.err != nil { + return true + } else if !iter.initialized || len(iter.nexttoken) != 0 { + iter.initialized = true + response, err := iter.paginate(iter.nexttoken) + iter.nextEventIndex = 0 + if err == nil { + iter.events = response.History.Events + iter.nexttoken = response.NextPageToken + iter.err = nil + } else { + iter.events = nil + iter.nexttoken = nil + iter.err = err + } + + if iter.nextEventIndex < len(iter.events) || iter.err != nil { + return true + } + return false + } + + return false +} + +func (iter *historyEventIteratorImpl) Next() (*historypb.HistoryEvent, error) { + // if caller call the Next() when iteration is over, just return nil, nil + if !iter.HasNext() { + panic("HistoryEventIterator Next() called without checking HasNext()") + } + + // we have cached events + if iter.nextEventIndex < len(iter.events) { + index := iter.nextEventIndex + iter.nextEventIndex++ + return iter.events[index], nil + } else if iter.err != nil { + // we have err, clear that iter.err and return err + err := iter.err + iter.err = nil + return nil, err + } + + panic("HistoryEventIterator Next() should return either a history event or a err") +} + +func (workflowRun *workflowRunImpl) GetRunID() string { + return workflowRun.currentRunID.Get() +} + +func (workflowRun *workflowRunImpl) GetID() string { + return workflowRun.workflowID +} + +func (workflowRun *workflowRunImpl) Get(ctx context.Context, valuePtr interface{}) error { + return workflowRun.GetWithOptions(ctx, valuePtr, WorkflowRunGetOptions{}) +} + +func (workflowRun *workflowRunImpl) GetWithOptions( + ctx context.Context, + valuePtr interface{}, + options WorkflowRunGetOptions, +) error { + iter := workflowRun.iterFn(ctx, workflowRun.currentRunID.Get()) + if !iter.HasNext() { + panic("could not get last history event for workflow") + } + closeEvent, err := iter.Next() + if err != nil { + return err + } + + switch closeEvent.GetEventType() { + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED: + attributes := closeEvent.GetWorkflowExecutionCompletedEventAttributes() + if !options.DisableFollowingRuns && attributes.NewExecutionRunId != "" { + return workflowRun.follow(ctx, valuePtr, attributes.NewExecutionRunId, options) + } + if valuePtr == nil || attributes.Result == nil { + return nil + } + rf := reflect.ValueOf(valuePtr) + if rf.Type().Kind() != reflect.Ptr { + return errors.New("value parameter is not a pointer") + } + return workflowRun.dataConverter.FromPayloads(attributes.Result, valuePtr) + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_FAILED: + attributes := closeEvent.GetWorkflowExecutionFailedEventAttributes() + if !options.DisableFollowingRuns && attributes.NewExecutionRunId != "" { + return workflowRun.follow(ctx, valuePtr, attributes.NewExecutionRunId, options) + } + err = workflowRun.failureConverter.FailureToError(attributes.GetFailure()) + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CANCELED: + attributes := closeEvent.GetWorkflowExecutionCanceledEventAttributes() + details := newEncodedValues(attributes.Details, workflowRun.dataConverter) + err = NewCanceledError(details) + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_TERMINATED: + err = newTerminatedError() + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_TIMED_OUT: + attributes := closeEvent.GetWorkflowExecutionTimedOutEventAttributes() + if !options.DisableFollowingRuns && attributes.NewExecutionRunId != "" { + return workflowRun.follow(ctx, valuePtr, attributes.NewExecutionRunId, options) + } + err = NewTimeoutError("Workflow timeout", enumspb.TIMEOUT_TYPE_START_TO_CLOSE, nil) + case enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW: + attributes := closeEvent.GetWorkflowExecutionContinuedAsNewEventAttributes() + if !options.DisableFollowingRuns { + return workflowRun.follow(ctx, valuePtr, attributes.NewExecutionRunId, options) + } + err := &ContinueAsNewError{ + WorkflowType: &WorkflowType{Name: attributes.GetWorkflowType().GetName()}, + Input: attributes.Input, + Header: attributes.Header, + TaskQueueName: attributes.GetTaskQueue().GetName(), + } + if attributes.WorkflowRunTimeout != nil { + err.WorkflowRunTimeout = attributes.WorkflowRunTimeout.AsDuration() + } + if attributes.WorkflowTaskTimeout != nil { + err.WorkflowTaskTimeout = attributes.WorkflowTaskTimeout.AsDuration() + } + return err + default: + return fmt.Errorf("unexpected event type %s when handling workflow execution result", closeEvent.GetEventType()) + } + + err = NewWorkflowExecutionError( + workflowRun.workflowID, + workflowRun.currentRunID.Get(), + workflowRun.workflowType, + err) + + return err +} + +// follow is used by Get to follow a chain of executions linked by NewExecutionRunId, so that Get +// doesn't return until the chain finishes. These can be ContinuedAsNew events, Completed events +// (for workflows with a cron schedule), or Failed or TimedOut events (for workflows with a retry +// policy or cron schedule). +func (workflowRun *workflowRunImpl) follow( + ctx context.Context, + valuePtr interface{}, + newRunID string, + options WorkflowRunGetOptions, +) error { + curRunID := util.PopulatedOnceCell(newRunID) + workflowRun.currentRunID = &curRunID + return workflowRun.GetWithOptions(ctx, valuePtr, options) +} + +func getWorkflowMemo(input map[string]interface{}, dc converter.DataConverter) (*commonpb.Memo, error) { + if input == nil { + return nil, nil + } + + memo := make(map[string]*commonpb.Payload) + for k, v := range input { + // TODO (shtin): use dc here??? + memoBytes, err := converter.GetDefaultDataConverter().ToPayload(v) + if err != nil { + return nil, fmt.Errorf("encode workflow memo error: %v", err.Error()) + } + memo[k] = memoBytes + } + return &commonpb.Memo{Fields: memo}, nil +} + +type workflowClientInterceptor struct { + client *WorkflowClient +} + +func (w *workflowClientInterceptor) ExecuteWorkflow( + ctx context.Context, + in *ClientExecuteWorkflowInput, +) (WorkflowRun, error) { + // This is always set before interceptor is invoked + workflowID := in.Options.ID + if workflowID == "" { + return nil, fmt.Errorf("no workflow ID in options") + } + + executionTimeout := in.Options.WorkflowExecutionTimeout + runTimeout := in.Options.WorkflowRunTimeout + workflowTaskTimeout := in.Options.WorkflowTaskTimeout + + dataConverter := WithContext(ctx, w.client.dataConverter) + if dataConverter == nil { + dataConverter = converter.GetDefaultDataConverter() + } + + // Encode input + input, err := encodeArgs(dataConverter, in.Args) + if err != nil { + return nil, err + } + + memo, err := getWorkflowMemo(in.Options.Memo, dataConverter) + if err != nil { + return nil, err + } + + searchAttr, err := serializeSearchAttributes(in.Options.SearchAttributes, in.Options.TypedSearchAttributes) + if err != nil { + return nil, err + } + + // get workflow headers from the context + header, err := headerPropagated(ctx, w.client.contextPropagators) + if err != nil { + return nil, err + } + + // run propagators to extract information about tracing and other stuff, store in headers field + startRequest := &workflowservice.StartWorkflowExecutionRequest{ + Namespace: w.client.namespace, + WorkflowId: workflowID, + WorkflowType: &commonpb.WorkflowType{Name: in.WorkflowType}, + TaskQueue: &taskqueuepb.TaskQueue{Name: in.Options.TaskQueue, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, + Input: input, + WorkflowExecutionTimeout: durationpb.New(executionTimeout), + WorkflowRunTimeout: durationpb.New(runTimeout), + WorkflowTaskTimeout: durationpb.New(workflowTaskTimeout), + Identity: w.client.identity, + WorkflowIdReusePolicy: in.Options.WorkflowIDReusePolicy, + WorkflowIdConflictPolicy: in.Options.WorkflowIDConflictPolicy, + RetryPolicy: convertToPBRetryPolicy(in.Options.RetryPolicy), + CronSchedule: in.Options.CronSchedule, + Memo: memo, + SearchAttributes: searchAttr, + Header: header, + CompletionCallbacks: in.Options.callbacks, + Links: in.Options.links, + } + + startRequest.UserMetadata, err = buildUserMetadata(in.Options.StaticSummary, in.Options.StaticDetails, dataConverter) + if err != nil { + return nil, err + } + + if in.Options.requestID != "" { + startRequest.RequestId = in.Options.requestID + } else { + startRequest.RequestId = uuid.New() + } + + var eagerExecutor *eagerWorkflowExecutor + if in.Options.EnableEagerStart && w.client.capabilities.GetEagerWorkflowStart() && w.client.eagerDispatcher != nil { + eagerExecutor = w.client.eagerDispatcher.applyToRequest(startRequest) + } + + if in.Options.StartDelay != 0 { + startRequest.WorkflowStartDelay = durationpb.New(in.Options.StartDelay) + } + + grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler( + w.client.metricsHandler.WithTags(metrics.RPCTags(in.WorkflowType, metrics.NoneTagValue, in.Options.TaskQueue))), + defaultGrpcRetryParameters(ctx)) + defer cancel() + + var runID string + if in.Options.WithStartOperation == nil { + response, err := w.client.workflowService.StartWorkflowExecution(grpcCtx, startRequest) + + eagerWorkflowTask := response.GetEagerWorkflowTask() + if eagerWorkflowTask != nil && eagerExecutor != nil { + eagerExecutor.handleResponse(eagerWorkflowTask) + } else if eagerExecutor != nil { + eagerExecutor.releaseUnused() + } + + // Allow already-started error + if e, ok := err.(*serviceerror.WorkflowExecutionAlreadyStarted); ok && !in.Options.WorkflowExecutionErrorWhenAlreadyStarted { + runID = e.RunId + } else if err != nil { + return nil, err + } else { + runID = response.RunId + } + } else { + response, err := w.executeWorkflowWithOperation(grpcCtx, startRequest, in.Options.WithStartOperation) + if err != nil { + return nil, err + } + runID = response.RunId + } + + iterFn := func(fnCtx context.Context, fnRunID string) HistoryEventIterator { + metricsHandler := w.client.metricsHandler.WithTags(metrics.RPCTags(in.WorkflowType, + metrics.NoneTagValue, in.Options.TaskQueue)) + return w.client.getWorkflowHistory(fnCtx, workflowID, fnRunID, true, + enumspb.HISTORY_EVENT_FILTER_TYPE_CLOSE_EVENT, metricsHandler) + } + + curRunIDCell := util.PopulatedOnceCell(runID) + return &workflowRunImpl{ + workflowType: in.WorkflowType, + workflowID: workflowID, + firstRunID: runID, + currentRunID: &curRunIDCell, + iterFn: iterFn, + dataConverter: w.client.dataConverter, + failureConverter: w.client.failureConverter, + registry: w.client.registry, + }, nil +} + +func (w *workflowClientInterceptor) executeWorkflowWithOperation( + ctx context.Context, + startRequest *workflowservice.StartWorkflowExecutionRequest, + operation WithStartWorkflowOperation, +) (*workflowservice.StartWorkflowExecutionResponse, error) { + startOp := &workflowservice.ExecuteMultiOperationRequest_Operation{ + Operation: &workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow{ + StartWorkflow: startRequest, + }, + } + + var withStartOp *workflowservice.ExecuteMultiOperationRequest_Operation + switch t := operation.(type) { + case *UpdateWithStartWorkflowOperation: + if err := t.markExecuted(); err != nil { + return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, err) + } + + if t.err != nil { + return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, t.err) + } + + updateReq, err := w.createUpdateWorkflowRequest(ctx, t.input) + if err != nil { + return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, err) + } + if updateReq.WorkflowExecution.WorkflowId == "" { + updateReq.WorkflowExecution.WorkflowId = startRequest.WorkflowId + } + + withStartOp = &workflowservice.ExecuteMultiOperationRequest_Operation{ + Operation: &workflowservice.ExecuteMultiOperationRequest_Operation_UpdateWorkflow{ + UpdateWorkflow: updateReq, + }, + } + default: + return nil, fmt.Errorf("%w: %T", errUnsupportedOperation, t) + } + + multiRequest := workflowservice.ExecuteMultiOperationRequest{ + Namespace: w.client.namespace, + Operations: []*workflowservice.ExecuteMultiOperationRequest_Operation{ + startOp, + withStartOp, + }, + } + multiResp, err := w.client.workflowService.ExecuteMultiOperation(ctx, &multiRequest) + + var multiErr *serviceerror.MultiOperationExecution + if errors.As(err, &multiErr) { + if len(multiErr.OperationErrors()) != len(multiRequest.Operations) { + return nil, fmt.Errorf("%w: %v instead of %v operation errors", + errInvalidServerResponse, len(multiErr.OperationErrors()), len(multiRequest.Operations)) + } + + var startErr error + var abortedErr *serviceerror.MultiOperationAborted + for i, opReq := range multiRequest.Operations { + // if an operation error is of type MultiOperationAborted, it means it was only aborted because + // of another operation's error and is therefore not interesting or helpful + opErr := multiErr.OperationErrors()[i] + + switch t := opReq.Operation.(type) { + case *workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow: + if !errors.As(opErr, &abortedErr) { + startErr = opErr + } + case *workflowservice.ExecuteMultiOperationRequest_Operation_UpdateWorkflow: + if !errors.As(opErr, &abortedErr) { + startErr = fmt.Errorf("%w: %w", errInvalidWorkflowOperation, opErr) + } + default: + // this would only happen if a case statement for a newly added operation is missing above + return nil, fmt.Errorf("%w: %T", errUnsupportedOperation, t) + } + } + return nil, startErr + } else if err != nil { + return nil, err + } + + if len(multiResp.Responses) != len(multiRequest.Operations) { + return nil, fmt.Errorf("%w: %v instead of %v operation results", + errInvalidServerResponse, len(multiResp.Responses), len(multiRequest.Operations)) + } + + var startResp *workflowservice.StartWorkflowExecutionResponse + for i, opReq := range multiRequest.Operations { + resp := multiResp.Responses[i].Response + + switch t := opReq.Operation.(type) { + case *workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow: + if opResp, ok := resp.(*workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow); ok { + startResp = opResp.StartWorkflow + } else { + return nil, fmt.Errorf("%w: StartWorkflow response has the wrong type %T", errInvalidServerResponse, resp) + } + case *workflowservice.ExecuteMultiOperationRequest_Operation_UpdateWorkflow: + if opResp, ok := resp.(*workflowservice.ExecuteMultiOperationResponse_Response_UpdateWorkflow); ok { + handle, err := w.updateHandleFromResponse( + ctx, + enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED, + opResp.UpdateWorkflow) + operation.(*UpdateWithStartWorkflowOperation).set(handle, err) + if err != nil { + return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, err) + } + } else { + return nil, fmt.Errorf("%w: UpdateWorkflow response has the wrong type %T", errInvalidServerResponse, resp) + } + default: + // this would only happen if a case statement for a newly added operation is missing above + return nil, fmt.Errorf("%w: %T", errUnsupportedOperation, t) + } + } + return startResp, nil +} + +func (w *workflowClientInterceptor) SignalWorkflow(ctx context.Context, in *ClientSignalWorkflowInput) error { + dataConverter := WithContext(ctx, w.client.dataConverter) + input, err := encodeArg(dataConverter, in.Arg) + if err != nil { + return err + } + + // get workflow headers from the context + header, err := headerPropagated(ctx, w.client.contextPropagators) + if err != nil { + return err + } + + request := &workflowservice.SignalWorkflowExecutionRequest{ + Namespace: w.client.namespace, + RequestId: uuid.New(), + WorkflowExecution: &commonpb.WorkflowExecution{ + WorkflowId: in.WorkflowID, + RunId: in.RunID, + }, + SignalName: in.SignalName, + Input: input, + Identity: w.client.identity, + Header: header, + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err = w.client.workflowService.SignalWorkflowExecution(grpcCtx, request) + return err +} + +func (w *workflowClientInterceptor) SignalWithStartWorkflow( + ctx context.Context, + in *ClientSignalWithStartWorkflowInput, +) (WorkflowRun, error) { + dataConverter := WithContext(ctx, w.client.dataConverter) + signalInput, err := encodeArg(dataConverter, in.SignalArg) + if err != nil { + return nil, err + } + + executionTimeout := in.Options.WorkflowExecutionTimeout + runTimeout := in.Options.WorkflowRunTimeout + taskTimeout := in.Options.WorkflowTaskTimeout + + // Encode input + input, err := encodeArgs(dataConverter, in.Args) + if err != nil { + return nil, err + } + + memo, err := getWorkflowMemo(in.Options.Memo, dataConverter) + if err != nil { + return nil, err + } + + searchAttr, err := serializeSearchAttributes(in.Options.SearchAttributes, in.Options.TypedSearchAttributes) + if err != nil { + return nil, err + } + + // get workflow headers from the context + header, err := headerPropagated(ctx, w.client.contextPropagators) + if err != nil { + return nil, err + } + + signalWithStartRequest := &workflowservice.SignalWithStartWorkflowExecutionRequest{ + Namespace: w.client.namespace, + RequestId: uuid.New(), + WorkflowId: in.Options.ID, + WorkflowType: &commonpb.WorkflowType{Name: in.WorkflowType}, + TaskQueue: &taskqueuepb.TaskQueue{Name: in.Options.TaskQueue, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, + Input: input, + WorkflowExecutionTimeout: durationpb.New(executionTimeout), + WorkflowRunTimeout: durationpb.New(runTimeout), + WorkflowTaskTimeout: durationpb.New(taskTimeout), + SignalName: in.SignalName, + SignalInput: signalInput, + Identity: w.client.identity, + RetryPolicy: convertToPBRetryPolicy(in.Options.RetryPolicy), + CronSchedule: in.Options.CronSchedule, + Memo: memo, + SearchAttributes: searchAttr, + WorkflowIdReusePolicy: in.Options.WorkflowIDReusePolicy, + WorkflowIdConflictPolicy: in.Options.WorkflowIDConflictPolicy, + Header: header, + } + + if in.Options.StartDelay != 0 { + signalWithStartRequest.WorkflowStartDelay = durationpb.New(in.Options.StartDelay) + } + + signalWithStartRequest.UserMetadata, err = buildUserMetadata(in.Options.StaticSummary, in.Options.StaticDetails, dataConverter) + if err != nil { + return nil, err + } + + var response *workflowservice.SignalWithStartWorkflowExecutionResponse + + // Start creating workflow request. + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + response, err = w.client.workflowService.SignalWithStartWorkflowExecution(grpcCtx, signalWithStartRequest) + if err != nil { + return nil, err + } + + iterFn := func(fnCtx context.Context, fnRunID string) HistoryEventIterator { + metricsHandler := w.client.metricsHandler.WithTags(metrics.RPCTags(in.WorkflowType, + metrics.NoneTagValue, in.Options.TaskQueue)) + return w.client.getWorkflowHistory(fnCtx, in.Options.ID, fnRunID, true, + enumspb.HISTORY_EVENT_FILTER_TYPE_CLOSE_EVENT, metricsHandler) + } + + curRunIDCell := util.PopulatedOnceCell(response.GetRunId()) + return &workflowRunImpl{ + workflowType: in.WorkflowType, + workflowID: in.Options.ID, + firstRunID: response.GetRunId(), + currentRunID: &curRunIDCell, + iterFn: iterFn, + dataConverter: w.client.dataConverter, + failureConverter: w.client.failureConverter, + registry: w.client.registry, + }, nil +} + +func (w *workflowClientInterceptor) CancelWorkflow(ctx context.Context, in *ClientCancelWorkflowInput) error { + request := &workflowservice.RequestCancelWorkflowExecutionRequest{ + Namespace: w.client.namespace, + RequestId: uuid.New(), + WorkflowExecution: &commonpb.WorkflowExecution{ + WorkflowId: in.WorkflowID, + RunId: in.RunID, + }, + Identity: w.client.identity, + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err := w.client.workflowService.RequestCancelWorkflowExecution(grpcCtx, request) + return err +} + +func (w *workflowClientInterceptor) TerminateWorkflow(ctx context.Context, in *ClientTerminateWorkflowInput) error { + datailsPayload, err := w.client.dataConverter.ToPayloads(in.Details...) + if err != nil { + return err + } + + request := &workflowservice.TerminateWorkflowExecutionRequest{ + Namespace: w.client.namespace, + WorkflowExecution: &commonpb.WorkflowExecution{ + WorkflowId: in.WorkflowID, + RunId: in.RunID, + }, + Reason: in.Reason, + Identity: w.client.identity, + Details: datailsPayload, + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + _, err = w.client.workflowService.TerminateWorkflowExecution(grpcCtx, request) + return err +} + +func (w *workflowClientInterceptor) QueryWorkflow( + ctx context.Context, + in *ClientQueryWorkflowInput, +) (converter.EncodedValue, error) { + // get workflow headers from the context + header, err := headerPropagated(ctx, w.client.contextPropagators) + if err != nil { + return nil, err + } + + var input *commonpb.Payloads + if len(in.Args) > 0 { + var err error + if input, err = encodeArgs(w.client.dataConverter, in.Args); err != nil { + return nil, err + } + } + req := &workflowservice.QueryWorkflowRequest{ + Namespace: w.client.namespace, + Execution: &commonpb.WorkflowExecution{ + WorkflowId: in.WorkflowID, + RunId: in.RunID, + }, + Query: &querypb.WorkflowQuery{ + QueryType: in.QueryType, + QueryArgs: input, + Header: header, + }, + QueryRejectCondition: in.QueryRejectCondition, + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + resp, err := w.client.workflowService.QueryWorkflow(grpcCtx, req) + if err != nil { + return nil, err + } + + if resp.QueryRejected != nil { + return nil, &QueryRejectedError{ + queryRejected: resp.QueryRejected, + } + } + return newEncodedValue(resp.QueryResult, w.client.dataConverter), nil +} + +func (w *workflowClientInterceptor) UpdateWorkflow( + ctx context.Context, + in *ClientUpdateWorkflowInput, +) (WorkflowUpdateHandle, error) { + var resp *workflowservice.UpdateWorkflowExecutionResponse + req, err := w.createUpdateWorkflowRequest(ctx, in) + if err != nil { + return nil, err + } + + for { + var err error + resp, err = func() (*workflowservice.UpdateWorkflowExecutionResponse, error) { + grpcCtx, cancel := newGRPCContext(ctx, grpcTimeout(pollUpdateTimeout), grpcLongPoll(true), defaultGrpcRetryParameters(ctx)) + defer cancel() + return w.client.workflowService.UpdateWorkflowExecution(grpcCtx, req) + }() + if err != nil { + if ctx.Err() != nil { + return nil, NewWorkflowUpdateServiceTimeoutOrCanceledError(err) + } + if status := serviceerror.ToStatus(err); status.Code() == codes.Canceled || status.Code() == codes.DeadlineExceeded { + return nil, NewWorkflowUpdateServiceTimeoutOrCanceledError(err) + } + return nil, err + } + // Once the update is past admitted we know it is durable + // Note: old server version may return UNSPECIFIED if the update request + // did not reach the desired lifecycle stage. + if resp.GetStage() != enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ADMITTED && + resp.GetStage() != enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED { + break + } + } + + // Here we know the update is at least accepted + desiredLifecycleStage := updateLifeCycleStageToProto(in.WaitForStage) + return w.updateHandleFromResponse(ctx, desiredLifecycleStage, resp) +} + +func createUpdateWorkflowInput( + options UpdateWorkflowOptions, +) (*ClientUpdateWorkflowInput, error) { + // Default update ID + updateID := options.UpdateID + if updateID == "" { + updateID = uuid.New() + } + + if options.WaitForStage == WorkflowUpdateStageUnspecified { + return nil, errors.New("WaitForStage must be specified") + } + + if options.WaitForStage == WorkflowUpdateStageAdmitted { + return nil, errors.New("WaitForStage WorkflowUpdateStageAdmitted is not supported") + } + + return &ClientUpdateWorkflowInput{ + UpdateID: updateID, + WorkflowID: options.WorkflowID, + UpdateName: options.UpdateName, + Args: options.Args, + RunID: options.RunID, + FirstExecutionRunID: options.FirstExecutionRunID, + WaitForStage: options.WaitForStage, + }, nil +} + +func (w *workflowClientInterceptor) createUpdateWorkflowRequest( + ctx context.Context, + in *ClientUpdateWorkflowInput, +) (*workflowservice.UpdateWorkflowExecutionRequest, error) { + argPayloads, err := w.client.dataConverter.ToPayloads(in.Args...) + if err != nil { + return nil, err + } + + header, err := headerPropagated(ctx, w.client.contextPropagators) + if err != nil { + return nil, err + } + + return &workflowservice.UpdateWorkflowExecutionRequest{ + WaitPolicy: &updatepb.WaitPolicy{LifecycleStage: updateLifeCycleStageToProto(in.WaitForStage)}, + Namespace: w.client.namespace, + WorkflowExecution: &commonpb.WorkflowExecution{ + WorkflowId: in.WorkflowID, + RunId: in.RunID, + }, + FirstExecutionRunId: in.FirstExecutionRunID, + Request: &updatepb.Request{ + Meta: &updatepb.Meta{ + UpdateId: in.UpdateID, + Identity: w.client.identity, + }, + Input: &updatepb.Input{ + Header: header, + Name: in.UpdateName, + Args: argPayloads, + }, + }, + }, nil +} + +func (w *workflowClientInterceptor) PollWorkflowUpdate( + parentCtx context.Context, + in *ClientPollWorkflowUpdateInput, +) (*ClientPollWorkflowUpdateOutput, error) { + // header, _ = headerPropagated(ctx, w.client.contextPropagators) + // todo header not in PollWorkflowUpdate + + pollReq := workflowservice.PollWorkflowExecutionUpdateRequest{ + Namespace: w.client.namespace, + UpdateRef: in.UpdateRef, + Identity: w.client.identity, + WaitPolicy: &updatepb.WaitPolicy{ + LifecycleStage: enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED, + }, + } + for { + ctx, cancel := newGRPCContext( + parentCtx, + grpcLongPoll(true), + grpcTimeout(pollUpdateTimeout), + ) + ctx = context.WithValue( + ctx, + retry.ConfigKey, + createDynamicServiceRetryPolicy(ctx).GrpcRetryConfig(), + ) + resp, err := w.client.workflowService.PollWorkflowExecutionUpdate(ctx, &pollReq) + cancel() + if err == nil && resp.GetOutcome() == nil { + continue + } + if err != nil { + if ctx.Err() != nil { + return nil, NewWorkflowUpdateServiceTimeoutOrCanceledError(err) + } + if code := status.Code(err); code == codes.Canceled || code == codes.DeadlineExceeded { + return nil, NewWorkflowUpdateServiceTimeoutOrCanceledError(err) + } + return nil, err + } + switch v := resp.GetOutcome().GetValue().(type) { + case *updatepb.Outcome_Failure: + return &ClientPollWorkflowUpdateOutput{ + Error: w.client.failureConverter.FailureToError(v.Failure), + }, nil + case *updatepb.Outcome_Success: + return &ClientPollWorkflowUpdateOutput{ + Result: newEncodedValue(v.Success, w.client.dataConverter), + }, nil + default: + return nil, fmt.Errorf("unsupported outcome type %T", v) + } + } +} + +// Required to implement ClientOutboundInterceptor +func (*workflowClientInterceptor) mustEmbedClientOutboundInterceptorBase() {} + +func (w *workflowClientInterceptor) updateHandleFromResponse( + ctx context.Context, + desiredLifecycleStage enumspb.UpdateWorkflowExecutionLifecycleStage, + resp *workflowservice.UpdateWorkflowExecutionResponse, +) (WorkflowUpdateHandle, error) { + if desiredLifecycleStage == enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED && + resp.GetStage() != enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED { + // TODO(https://github.com/temporalio/features/issues/428) replace with handle wait for stage once implemented + pollResp, err := w.client.PollWorkflowUpdate(ctx, resp.GetUpdateRef()) + if err != nil { + return nil, err + } + if pollResp.Error != nil { + return &completedUpdateHandle{ + err: pollResp.Error, + baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, + }, nil + } else { + return &completedUpdateHandle{ + value: pollResp.Result, + baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, + }, nil + } + } + + switch v := resp.GetOutcome().GetValue().(type) { + case nil: + return &lazyUpdateHandle{ + client: w.client, + baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, + }, nil + case *updatepb.Outcome_Failure: + return &completedUpdateHandle{ + err: w.client.failureConverter.FailureToError(v.Failure), + baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, + }, nil + case *updatepb.Outcome_Success: + return &completedUpdateHandle{ + value: newEncodedValue(v.Success, w.client.dataConverter), + baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, + }, nil + } + return nil, fmt.Errorf("unsupported outcome type %T", resp.GetOutcome().GetValue()) +} + +func (uh *baseUpdateHandle) WorkflowID() string { + return uh.ref.GetWorkflowExecution().GetWorkflowId() +} + +func (uh *baseUpdateHandle) RunID() string { + return uh.ref.GetWorkflowExecution().GetRunId() +} + +func (uh *baseUpdateHandle) UpdateID() string { + return uh.ref.GetUpdateId() +} + +func (ch *completedUpdateHandle) Get(ctx context.Context, valuePtr interface{}) error { + if ch.err != nil || valuePtr == nil { + return ch.err + } + if err := ch.value.Get(valuePtr); err != nil { + return err + } + return nil +} + +func (luh *lazyUpdateHandle) Get(ctx context.Context, valuePtr interface{}) error { + resp, err := luh.client.PollWorkflowUpdate(ctx, luh.ref) + if err != nil { + return err + } + if resp.Error != nil || valuePtr == nil { + return resp.Error + } + return resp.Result.Get(valuePtr) +} + +func (q *QueryRejectedError) QueryRejected() *querypb.QueryRejected { + return q.queryRejected +} + +func (q *QueryRejectedError) Error() string { + return fmt.Sprintf("query rejected: %s", q.queryRejected.Status.String()) +} + +func buildUserMetadata( + summary string, + details string, + dataConverter converter.DataConverter, +) (*sdk.UserMetadata, error) { + if summary == "" && details == "" { + return nil, nil + } + ret := &sdk.UserMetadata{} + var err error + if summary != "" { + if ret.Summary, err = dataConverter.ToPayload(summary); err != nil { + return nil, fmt.Errorf("failed converting summary to payload: %w", err) + } + } + if details != "" { + if ret.Details, err = dataConverter.ToPayload(details); err != nil { + return nil, fmt.Errorf("failed converting details to payload: %w", err) + } + } + return ret, nil +} diff --git a/vendor/go.temporal.io/sdk/internal/internal_workflow_testsuite.go b/vendor/go.temporal.io/sdk/internal/internal_workflow_testsuite.go new file mode 100644 index 00000000000..fc7f119b532 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/internal_workflow_testsuite.go @@ -0,0 +1,2921 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "errors" + "fmt" + "reflect" + "strconv" + "strings" + "sync" + "time" + + "github.com/facebookgo/clock" + "github.com/golang/mock/gomock" + "github.com/google/uuid" + "github.com/nexus-rpc/sdk-go/nexus" + "github.com/robfig/cron" + "github.com/stretchr/testify/mock" + "google.golang.org/grpc" + "google.golang.org/protobuf/types/known/durationpb" + "google.golang.org/protobuf/types/known/timestamppb" + + commandpb "go.temporal.io/api/command/v1" + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + failurepb "go.temporal.io/api/failure/v1" + nexuspb "go.temporal.io/api/nexus/v1" + "go.temporal.io/api/serviceerror" + taskqueuepb "go.temporal.io/api/taskqueue/v1" + "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/api/workflowservicemock/v1" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" + ilog "go.temporal.io/sdk/internal/log" + "go.temporal.io/sdk/log" +) + +const ( + defaultTestNamespace = "default-test-namespace" + defaultTestTaskQueue = "default-test-taskqueue" + defaultTestWorkflowID = "default-test-workflow-id" + defaultTestRunID = "default-test-run-id" + defaultTestWorkflowTypeName = "default-test-workflow-type-name" + workflowTypeNotSpecified = "workflow-type-not-specified" + + // These are copied from service implementation + reservedTaskQueuePrefix = "/__temporal_sys/" + maxIDLengthLimit = 1000 + maxWorkflowTimeout = 24 * time.Hour * 365 * 10 + + defaultMaximumAttemptsForUnitTest = 10 +) + +type ( + testTimerHandle struct { + env *testWorkflowEnvironmentImpl + callback ResultHandler + timer *clock.Timer + wallTimer *clock.Timer + duration time.Duration + mockTimeToFire time.Time + wallTimeToFire time.Time + timerID int64 + } + + testActivityHandle struct { + callback ResultHandler + activityType string + heartbeatDetails *commonpb.Payloads + } + + testWorkflowHandle struct { + env *testWorkflowEnvironmentImpl + callback ResultHandler + handled bool + params *ExecuteWorkflowParams + err error + } + + testNexusOperationHandle struct { + env *testWorkflowEnvironmentImpl + seq int64 + params executeNexusOperationParams + operationID string + cancelRequested bool + started bool + done bool + onCompleted func(*commonpb.Payload, error) + onStarted func(opID string, e error) + } + + testCallbackHandle struct { + callback func() + startWorkflowTask bool // start a new workflow task after callback() is handled. + env *testWorkflowEnvironmentImpl + } + + activityExecutorWrapper struct { + *activityExecutor + env *testWorkflowEnvironmentImpl + } + + workflowExecutorWrapper struct { + *workflowExecutor + env *testWorkflowEnvironmentImpl + } + + mockWrapper struct { + env *testWorkflowEnvironmentImpl + name string + fn interface{} + isWorkflow bool + dataConverter converter.DataConverter + } + + taskQueueSpecificActivity struct { + fn interface{} + taskQueues map[string]struct{} + } + + // testWorkflowEnvironmentShared is the shared data between parent workflow and child workflow test environments + testWorkflowEnvironmentShared struct { + locker sync.Mutex + testSuite *WorkflowTestSuite + + taskQueueSpecificActivities map[string]*taskQueueSpecificActivity + + workflowMock *mock.Mock + activityMock *mock.Mock + service workflowservice.WorkflowServiceClient + logger log.Logger + metricsHandler metrics.Handler + contextPropagators []ContextPropagator + identity string + detachedChildWaitDisabled bool + + mockClock *clock.Mock + wallClock clock.Clock + + callbackChannel chan testCallbackHandle + testTimeout time.Duration + header *commonpb.Header + + counterID int64 + activities map[string]*testActivityHandle + localActivities map[string]*localActivityTask + timers map[string]*testTimerHandle + runningWorkflows map[string]*testWorkflowHandle + runningNexusOperations map[int64]*testNexusOperationHandle + + runningCount int + + expectedWorkflowMockCalls map[string]struct{} + expectedActivityMockCalls map[string]struct{} + + onActivityStartedListener func(activityInfo *ActivityInfo, ctx context.Context, args converter.EncodedValues) + onActivityCompletedListener func(activityInfo *ActivityInfo, result converter.EncodedValue, err error) + onActivityCanceledListener func(activityInfo *ActivityInfo) + onLocalActivityStartedListener func(activityInfo *ActivityInfo, ctx context.Context, args []interface{}) + onLocalActivityCompletedListener func(activityInfo *ActivityInfo, result converter.EncodedValue, err error) + onLocalActivityCanceledListener func(activityInfo *ActivityInfo) + onActivityHeartbeatListener func(activityInfo *ActivityInfo, details converter.EncodedValues) + onChildWorkflowStartedListener func(workflowInfo *WorkflowInfo, ctx Context, args converter.EncodedValues) + onChildWorkflowCompletedListener func(workflowInfo *WorkflowInfo, result converter.EncodedValue, err error) + onChildWorkflowCanceledListener func(workflowInfo *WorkflowInfo) + onTimerScheduledListener func(timerID string, duration time.Duration) + onTimerFiredListener func(timerID string) + onTimerCanceledListener func(timerID string) + } + + // testWorkflowEnvironmentImpl is the environment that runs the workflow/activity unit tests. + testWorkflowEnvironmentImpl struct { + *testWorkflowEnvironmentShared + parentEnv *testWorkflowEnvironmentImpl + registry *registry + + workflowInfo *WorkflowInfo + workflowDef WorkflowDefinition + changeVersions map[string]Version + openSessions map[string]*SessionInfo + + workflowCancelHandler func() + signalHandler func(name string, input *commonpb.Payloads, header *commonpb.Header) error + queryHandler func(string, *commonpb.Payloads, *commonpb.Header) (*commonpb.Payloads, error) + updateHandler func(name string, id string, input *commonpb.Payloads, header *commonpb.Header, resp UpdateCallbacks) + startedHandler func(r WorkflowExecution, e error) + + isWorkflowCompleted bool + testResult converter.EncodedValue + testError error + doneChannel chan struct{} + workerOptions WorkerOptions + dataConverter converter.DataConverter + failureConverter converter.FailureConverter + runTimeout time.Duration + + heartbeatDetails *commonpb.Payloads + + workerStopChannel chan struct{} + sessionEnvironment *testSessionEnvironmentImpl + + // True if this was created only for testing activities not workflows. + activityEnvOnly bool + + workflowFunctionExecuting bool + bufferedUpdateRequests map[string][]func() + } + + testSessionEnvironmentImpl struct { + *sessionEnvironmentImpl + testWorkflowEnvironment *testWorkflowEnvironmentImpl + } +) + +func newTestWorkflowEnvironmentImpl(s *WorkflowTestSuite, parentRegistry *registry) *testWorkflowEnvironmentImpl { + var r *registry + if parentRegistry == nil { + r = newRegistryWithOptions(registryOptions{disableAliasing: s.disableRegistrationAliasing}) + } else { + r = parentRegistry + } + + env := &testWorkflowEnvironmentImpl{ + testWorkflowEnvironmentShared: &testWorkflowEnvironmentShared{ + testSuite: s, + taskQueueSpecificActivities: make(map[string]*taskQueueSpecificActivity), + + logger: s.logger, + metricsHandler: s.metricsHandler, + mockClock: clock.NewMock(), + wallClock: clock.New(), + timers: make(map[string]*testTimerHandle), + activities: make(map[string]*testActivityHandle), + localActivities: make(map[string]*localActivityTask), + runningWorkflows: make(map[string]*testWorkflowHandle), + runningNexusOperations: make(map[int64]*testNexusOperationHandle), + callbackChannel: make(chan testCallbackHandle, 1000), + testTimeout: 3 * time.Second, + expectedWorkflowMockCalls: make(map[string]struct{}), + expectedActivityMockCalls: make(map[string]struct{}), + }, + + workflowInfo: &WorkflowInfo{ + Namespace: defaultTestNamespace, + WorkflowExecution: WorkflowExecution{ + ID: defaultTestWorkflowID, + RunID: defaultTestRunID, + }, + WorkflowType: WorkflowType{Name: workflowTypeNotSpecified}, + TaskQueueName: defaultTestTaskQueue, + + WorkflowExecutionTimeout: maxWorkflowTimeout, + WorkflowTaskTimeout: 1 * time.Second, + Attempt: 1, + }, + registry: r, + + changeVersions: make(map[string]Version), + openSessions: make(map[string]*SessionInfo), + + doneChannel: make(chan struct{}), + workerStopChannel: make(chan struct{}), + dataConverter: converter.GetDefaultDataConverter(), + failureConverter: GetDefaultFailureConverter(), + runTimeout: maxWorkflowTimeout, + bufferedUpdateRequests: make(map[string][]func()), + } + + if debugMode { + env.testTimeout = time.Hour * 24 + env.workerOptions.DeadlockDetectionTimeout = unlimitedDeadlockDetectionTimeout + } + + // move forward the mock clock to start time. + env.setStartTime(time.Now()) + + // put current workflow as a running workflow so child can send signal to parent + env.runningWorkflows[env.workflowInfo.WorkflowExecution.ID] = &testWorkflowHandle{env: env, callback: func(result *commonpb.Payloads, err error) {}} + + if env.logger == nil { + env.logger = ilog.NewDefaultLogger() + } + if env.metricsHandler == nil { + env.metricsHandler = metrics.NopHandler + } + env.contextPropagators = s.contextPropagators + env.header = s.header + + // setup mock service + mockCtrl := gomock.NewController(ilog.NewTestReporter(env.logger)) + mockService := workflowservicemock.NewMockWorkflowServiceClient(mockCtrl) + + mockHeartbeatFn := func(c context.Context, r *workflowservice.RecordActivityTaskHeartbeatRequest, opts ...grpc.CallOption) error { + activityID := ActivityID{id: string(r.TaskToken)} + env.locker.Lock() // need lock as this is running in activity worker's goroutinue + activityHandle, ok := env.getActivityHandle(activityID.id, GetActivityInfo(c).WorkflowExecution.RunID) + env.locker.Unlock() + if !ok { + env.logger.Debug("RecordActivityTaskHeartbeat: ActivityID not found, could be already completed or canceled.", + tagActivityID, activityID) + return serviceerror.NewNotFound("") + } + activityHandle.heartbeatDetails = r.Details + activityInfo := env.getActivityInfo(activityID, activityHandle.activityType) + if env.onActivityHeartbeatListener != nil { + // If we're only in an activity environment, posted callbacks are not + // invoked + if env.activityEnvOnly { + env.onActivityHeartbeatListener(activityInfo, newEncodedValues(r.Details, env.GetDataConverter())) + } else { + env.postCallback(func() { + env.onActivityHeartbeatListener(activityInfo, newEncodedValues(r.Details, env.GetDataConverter())) + }, false) + } + } + + env.logger.Debug("RecordActivityTaskHeartbeat", tagActivityID, activityID) + return nil + } + + mockService.EXPECT().RecordActivityTaskHeartbeat(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(func( + ctx context.Context, + r *workflowservice.RecordActivityTaskHeartbeatRequest, + opts ...grpc.CallOption, + ) (*workflowservice.RecordActivityTaskHeartbeatResponse, error) { + if err := mockHeartbeatFn(ctx, r, opts...); err != nil { + return nil, err + } + return &workflowservice.RecordActivityTaskHeartbeatResponse{CancelRequested: false}, nil + }).AnyTimes() + + env.service = mockService + + return env +} + +func (env *testWorkflowEnvironmentImpl) setStartTime(startTime time.Time) { + // move forward the mock clock to start time. + if startTime.IsZero() { + // if start time not set, use current clock time + startTime = env.wallClock.Now() + } + env.mockClock.Add(startTime.Sub(env.mockClock.Now())) + env.workflowInfo.WorkflowStartTime = env.mockClock.Now() +} + +func (env *testWorkflowEnvironmentImpl) setCurrentHistoryLength(length int) { + env.workflowInfo.currentHistoryLength = length +} + +func (env *testWorkflowEnvironmentImpl) setCurrentHistorySize(size int) { + env.workflowInfo.currentHistorySize = size +} + +func (env *testWorkflowEnvironmentImpl) setContinueAsNewSuggested(suggest bool) { + env.workflowInfo.continueAsNewSuggested = suggest +} + +func (env *testWorkflowEnvironmentImpl) setContinuedExecutionRunID(rid string) { + env.workflowInfo.ContinuedExecutionRunID = rid +} + +func (env *testWorkflowEnvironmentImpl) newTestWorkflowEnvironmentForChild(params *ExecuteWorkflowParams, callback ResultHandler, startedHandler func(r WorkflowExecution, e error)) (*testWorkflowEnvironmentImpl, error) { + // create a new test env + childEnv := newTestWorkflowEnvironmentImpl(env.testSuite, env.registry) + childEnv.parentEnv = env + childEnv.startedHandler = startedHandler + childEnv.testWorkflowEnvironmentShared = env.testWorkflowEnvironmentShared + childEnv.workerOptions = env.workerOptions + childEnv.dataConverter = params.DataConverter + childEnv.failureConverter = env.failureConverter + childEnv.registry = env.registry + childEnv.detachedChildWaitDisabled = env.detachedChildWaitDisabled + + if params.TaskQueueName == "" { + return nil, serviceerror.NewWorkflowExecutionAlreadyStarted("Empty task queue name", "", "") + } + + if params.WorkflowID == "" { + params.WorkflowID = env.workflowInfo.WorkflowExecution.RunID + "_" + getStringID(env.nextID()) + } + var cronSchedule string + if len(params.CronSchedule) > 0 { + cronSchedule = params.CronSchedule + } + // set workflow info data for child workflow + childEnv.header = params.Header + childEnv.workflowInfo.Attempt = params.attempt + childEnv.workflowInfo.WorkflowExecution.ID = params.WorkflowID + childEnv.workflowInfo.WorkflowExecution.RunID = params.WorkflowID + "_RunID" + childEnv.workflowInfo.Namespace = params.Namespace + childEnv.workflowInfo.TaskQueueName = params.TaskQueueName + childEnv.workflowInfo.WorkflowExecutionTimeout = params.WorkflowExecutionTimeout + childEnv.workflowInfo.WorkflowRunTimeout = params.WorkflowRunTimeout + childEnv.workflowInfo.WorkflowTaskTimeout = params.WorkflowTaskTimeout + childEnv.workflowInfo.lastCompletionResult = params.lastCompletionResult + childEnv.workflowInfo.CronSchedule = cronSchedule + childEnv.workflowInfo.ParentWorkflowNamespace = env.workflowInfo.Namespace + childEnv.workflowInfo.ParentWorkflowExecution = &env.workflowInfo.WorkflowExecution + + searchAttrs, err := serializeSearchAttributes(params.SearchAttributes, params.TypedSearchAttributes) + if err != nil { + return nil, err + } + childEnv.workflowInfo.SearchAttributes = searchAttrs + + childEnv.runTimeout = params.WorkflowRunTimeout + if workflowHandler, ok := env.runningWorkflows[params.WorkflowID]; ok { + // duplicate workflow ID + if !workflowHandler.handled { + return nil, serviceerror.NewWorkflowExecutionAlreadyStarted("Workflow execution already started", "", "") + } + if params.WorkflowIDReusePolicy == enumspb.WORKFLOW_ID_REUSE_POLICY_REJECT_DUPLICATE { + return nil, serviceerror.NewWorkflowExecutionAlreadyStarted("Workflow execution already started", "", "") + } + if workflowHandler.err == nil && params.WorkflowIDReusePolicy == enumspb.WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY { + return nil, serviceerror.NewWorkflowExecutionAlreadyStarted("Workflow execution already started", "", "") + } + } + + env.runningWorkflows[params.WorkflowID] = &testWorkflowHandle{env: childEnv, callback: callback, params: params} + + return childEnv, nil +} + +func (env *testWorkflowEnvironmentImpl) setWorkerOptions(options WorkerOptions) { + env.workerOptions = options + env.registry.interceptors = options.Interceptors + if env.workerOptions.EnableSessionWorker && env.sessionEnvironment == nil { + env.registry.RegisterActivityWithOptions(sessionCreationActivity, RegisterActivityOptions{ + Name: sessionCreationActivityName, + DisableAlreadyRegisteredCheck: true, + }) + env.registry.RegisterActivityWithOptions(sessionCompletionActivity, RegisterActivityOptions{ + Name: sessionCompletionActivityName, + DisableAlreadyRegisteredCheck: true, + }) + } +} + +func (env *testWorkflowEnvironmentImpl) setIdentity(identity string) { + env.identity = identity +} + +func (env *testWorkflowEnvironmentImpl) setDataConverter(dataConverter converter.DataConverter) { + env.dataConverter = dataConverter +} + +func (env *testWorkflowEnvironmentImpl) setFailureConverter(failureConverter converter.FailureConverter) { + env.failureConverter = failureConverter +} + +func (env *testWorkflowEnvironmentImpl) setContextPropagators(contextPropagators []ContextPropagator) { + env.contextPropagators = contextPropagators +} + +func (env *testWorkflowEnvironmentImpl) setWorkerStopChannel(c chan struct{}) { + env.workerStopChannel = c +} + +func (env *testWorkflowEnvironmentImpl) setDetachedChildWaitDisabled(detachedChildWaitDisabled bool) { + env.detachedChildWaitDisabled = detachedChildWaitDisabled +} + +func (env *testWorkflowEnvironmentImpl) setActivityTaskQueue(taskqueue string, activityFns ...interface{}) { + for _, activityFn := range activityFns { + fnName := getActivityFunctionName(env.registry, activityFn) + taskQueueActivity, ok := env.taskQueueSpecificActivities[fnName] + if !ok { + taskQueueActivity = &taskQueueSpecificActivity{fn: activityFn, taskQueues: make(map[string]struct{})} + env.taskQueueSpecificActivities[fnName] = taskQueueActivity + } + taskQueueActivity.taskQueues[taskqueue] = struct{}{} + } +} + +func (env *testWorkflowEnvironmentImpl) executeWorkflow(workflowFn interface{}, args ...interface{}) { + fType := reflect.TypeOf(workflowFn) + if getKind(fType) == reflect.Func { + env.RegisterWorkflowWithOptions(workflowFn, RegisterWorkflowOptions{DisableAlreadyRegisteredCheck: true}) + } + workflowType, input, err := getValidatedWorkflowFunction(workflowFn, args, env.GetDataConverter(), env.GetRegistry()) + if err != nil { + panic(err) + } + env.executeWorkflowInternal(0, workflowType.Name, input) +} + +func (env *testWorkflowEnvironmentImpl) executeWorkflowInternal(delayStart time.Duration, workflowType string, input *commonpb.Payloads) { + env.locker.Lock() + wInfo := env.workflowInfo + if wInfo.WorkflowType.Name != workflowTypeNotSpecified { + // Current TestWorkflowEnvironment only support to run one workflow. + // Created task to support testing multiple workflows with one env instance + // https://github.com/temporalio/go-sdk/issues/50 + panic(fmt.Sprintf("Current TestWorkflowEnvironment is used to execute %v. Please create a new TestWorkflowEnvironment for %v.", wInfo.WorkflowType.Name, workflowType)) + } + wInfo.WorkflowType.Name = workflowType + if wInfo.WorkflowRunTimeout == 0 { + wInfo.WorkflowRunTimeout = env.runTimeout + } + if wInfo.WorkflowExecutionTimeout == 0 { + wInfo.WorkflowExecutionTimeout = maxWorkflowTimeout + } + if wInfo.WorkflowTaskTimeout == 0 { + wInfo.WorkflowTaskTimeout = 1 * time.Second + } + env.locker.Unlock() + + workflowDefinition, err := env.getWorkflowDefinition(wInfo.WorkflowType) + if err != nil { + panic(err) + } + env.workflowDef = workflowDefinition + + // env.workflowDef.Execute() method will execute dispatcher. We want the dispatcher to only run in main loop. + // In case of child workflow, this executeWorkflowInternal() is run in separate goroutinue, so use postCallback + // to make sure workflowDef.Execute() is run in main loop. + env.postCallback(func() { + env.workflowDef.Execute(env, env.header, input) + // kick off first workflow task to start the workflow + if delayStart == 0 { + env.startWorkflowTask() + } else { + // we need to delayStart start workflow, decrease runningCount so mockClock could auto forward + env.runningCount-- + env.registerDelayedCallback(func() { + env.runningCount++ + env.startWorkflowTask() + }, delayStart) + } + }, false) + + if env.runTimeout > 0 { + timeoutDuration := env.runTimeout + delayStart + env.registerDelayedCallback(func() { + if !env.isWorkflowCompleted { + env.Complete(nil, ErrDeadlineExceeded) + } + }, timeoutDuration) + } + env.startMainLoop() +} + +func (env *testWorkflowEnvironmentImpl) getWorkflowDefinition(wt WorkflowType) (WorkflowDefinition, error) { + wf, ok := env.registry.getWorkflowFn(wt.Name) + if !ok { + supported := strings.Join(env.registry.getRegisteredWorkflowTypes(), ", ") + return nil, fmt.Errorf("unable to find workflow type: %v. Supported types: [%v]", wt.Name, supported) + } + wd := &workflowExecutorWrapper{ + workflowExecutor: &workflowExecutor{workflowType: wt.Name, fn: wf, interceptors: env.registry.interceptors}, + env: env, + } + return newSyncWorkflowDefinition(wd), nil +} + +func (env *testWorkflowEnvironmentImpl) TryUse(flag sdkFlag) bool { + return true +} + +func (env *testWorkflowEnvironmentImpl) QueueUpdate(name string, f func()) { + env.bufferedUpdateRequests[name] = append(env.bufferedUpdateRequests[name], f) +} + +func (env *testWorkflowEnvironmentImpl) HandleQueuedUpdates(name string) { + if bufferedUpdateRequests, ok := env.bufferedUpdateRequests[name]; ok { + for _, requests := range bufferedUpdateRequests { + requests() + } + delete(env.bufferedUpdateRequests, name) + } +} + +func (env *testWorkflowEnvironmentImpl) DrainUnhandledUpdates() bool { + // Due to mock registration the test environment cannot run the workflow function + // in the first "workflow task". We need to delay the draining until the main function has + // had a chance to run. + if !env.workflowFunctionExecuting { + return false + } + anyExecuted := false + // Check if any buffered update requests remain when we have no more coroutines to run and let them schedule so they are rejected. + // Generally iterating a map in workflow code is bad because it is non deterministic + // this case is fine since all these update handles will be rejected and not recorded in history. + for name, bufferedUpdateRequests := range env.bufferedUpdateRequests { + for _, request := range bufferedUpdateRequests { + request() + anyExecuted = true + } + delete(env.bufferedUpdateRequests, name) + } + return anyExecuted +} + +func (env *testWorkflowEnvironmentImpl) executeActivity( + activityFn interface{}, + args ...interface{}, +) (converter.EncodedValue, error) { + activityType, err := getValidatedActivityFunction(activityFn, args, env.registry) + if err != nil { + panic(err) + } + + input, err := encodeArgs(env.GetDataConverter(), args) + if err != nil { + panic(err) + } + + parameters := ExecuteActivityParams{ + ExecuteActivityOptions: ExecuteActivityOptions{ + ScheduleToCloseTimeout: 600 * time.Second, + StartToCloseTimeout: 600 * time.Second, + }, + ActivityType: *activityType, + Input: input, + Header: env.header, + } + + scheduleTaskAttr := &commandpb.ScheduleActivityTaskCommandAttributes{} + if parameters.ActivityID == "" { + scheduleTaskAttr.ActivityId = getStringID(env.nextID()) + } else { + scheduleTaskAttr.ActivityId = parameters.ActivityID + } + scheduleTaskAttr.ActivityType = &commonpb.ActivityType{Name: parameters.ActivityType.Name} + scheduleTaskAttr.TaskQueue = &taskqueuepb.TaskQueue{Name: parameters.TaskQueueName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} + scheduleTaskAttr.Input = parameters.Input + scheduleTaskAttr.ScheduleToCloseTimeout = durationpb.New(parameters.ScheduleToCloseTimeout) + scheduleTaskAttr.StartToCloseTimeout = durationpb.New(parameters.StartToCloseTimeout) + scheduleTaskAttr.ScheduleToStartTimeout = durationpb.New(parameters.ScheduleToStartTimeout) + scheduleTaskAttr.HeartbeatTimeout = durationpb.New(parameters.HeartbeatTimeout) + scheduleTaskAttr.RetryPolicy = parameters.RetryPolicy + scheduleTaskAttr.Header = parameters.Header + + workflowType := env.workflowInfo.WorkflowType.Name + if workflowType == workflowTypeNotSpecified { + workflowType = "0" + } + task := newTestActivityTask( + env.workflowInfo.WorkflowExecution.ID, + env.workflowInfo.WorkflowExecution.RunID, + workflowType, + env.workflowInfo.Namespace, + scheduleTaskAttr, + ) + + task.HeartbeatDetails = env.heartbeatDetails + + // ensure activityFn is registered to defaultTestTaskQueue + taskHandler := env.newTestActivityTaskHandler(defaultTestTaskQueue, env.GetDataConverter()) + activityHandle := &testActivityHandle{callback: func(result *commonpb.Payloads, err error) {}, activityType: parameters.ActivityType.Name} + activityID := ActivityID{id: scheduleTaskAttr.GetActivityId()} + env.setActivityHandle(activityID.id, env.workflowInfo.WorkflowExecution.RunID, activityHandle) + + result, err := taskHandler.Execute(defaultTestTaskQueue, task) + if err != nil { + if err == context.DeadlineExceeded { + env.logger.Debug(fmt.Sprintf("Activity %v timed out", task.ActivityType.Name)) + return nil, env.wrapActivityError(activityID, scheduleTaskAttr.ActivityType.Name, enumspb.RETRY_STATE_TIMEOUT, NewTimeoutError("Activity timeout", enumspb.TIMEOUT_TYPE_START_TO_CLOSE, err)) + } + topLine := fmt.Sprintf("activity for %s [panic]:", defaultTestTaskQueue) + st := getStackTraceRaw(topLine, 7, 0) + return nil, env.wrapActivityError(activityID, scheduleTaskAttr.ActivityType.Name, enumspb.RETRY_STATE_UNSPECIFIED, newPanicError(err.Error(), st)) + } + + if result == ErrActivityResultPending { + return nil, ErrActivityResultPending + } + + switch request := result.(type) { + case *workflowservice.RespondActivityTaskCanceledRequest: + details := newEncodedValues(request.Details, env.GetDataConverter()) + return nil, env.wrapActivityError(activityID, scheduleTaskAttr.ActivityType.Name, enumspb.RETRY_STATE_NON_RETRYABLE_FAILURE, NewCanceledError(details)) + case *workflowservice.RespondActivityTaskFailedRequest: + return nil, env.wrapActivityError(activityID, scheduleTaskAttr.ActivityType.Name, enumspb.RETRY_STATE_UNSPECIFIED, env.GetFailureConverter().FailureToError(request.GetFailure())) + case *workflowservice.RespondActivityTaskCompletedRequest: + return newEncodedValue(request.Result, env.GetDataConverter()), nil + default: + // will never happen + return nil, fmt.Errorf("unsupported respond type %T", result) + } +} + +func (env *testWorkflowEnvironmentImpl) executeLocalActivity( + activityFn interface{}, + args ...interface{}, +) (val converter.EncodedValue, err error) { + params := ExecuteLocalActivityParams{ + ExecuteLocalActivityOptions: ExecuteLocalActivityOptions{ + ScheduleToCloseTimeout: env.testTimeout, + }, + ActivityFn: activityFn, + InputArgs: args, + WorkflowInfo: env.workflowInfo, + Header: env.header, + } + task := &localActivityTask{ + activityID: "test-local-activity", + params: ¶ms, + callback: func(lar *LocalActivityResultWrapper) { + }, + attempt: 1, + scheduledTime: time.Now(), + header: params.Header, + } + taskHandler := localActivityTaskHandler{ + userContext: env.workerOptions.BackgroundActivityContext, + metricsHandler: env.metricsHandler, + logger: env.logger, + interceptors: env.registry.interceptors, + contextPropagators: env.contextPropagators, + } + + result := taskHandler.executeLocalActivityTask(task) + if result.err != nil { + activityType, _ := getValidatedActivityFunction(activityFn, args, env.registry) + return nil, env.wrapActivityError(ActivityID{id: task.activityID}, activityType.Name, enumspb.RETRY_STATE_UNSPECIFIED, result.err) + } + return newEncodedValue(result.result, env.GetDataConverter()), nil +} + +func (env *testWorkflowEnvironmentImpl) startWorkflowTask() { + if !env.isWorkflowCompleted { + env.workflowDef.OnWorkflowTaskStarted(env.workerOptions.DeadlockDetectionTimeout) + } +} + +func (env *testWorkflowEnvironmentImpl) isChildWorkflow() bool { + return env.parentEnv != nil +} + +func (env *testWorkflowEnvironmentImpl) startMainLoop() { + if env.isChildWorkflow() { + // child workflow rely on parent workflow's main loop to process events + <-env.doneChannel // wait until workflow is complete + return + } + + // notify all child workflows to exit their main loop + defer close(env.doneChannel) + + for !env.shouldStopEventLoop() { + // use non-blocking-select to check if there is anything pending in the main thread. + select { + case c := <-env.callbackChannel: + // this will drain the callbackChannel + c.processCallback() + default: + // nothing to process, main thread is blocked at this moment, now check if we should auto fire next timer + if !env.autoFireNextTimer() { + if env.shouldStopEventLoop() { + return + } + + // no timer to fire, wait for things to do or timeout. + select { + case c := <-env.callbackChannel: + c.processCallback() + case <-time.After(env.testTimeout): + // not able to complete workflow within test timeout, workflow likely stuck somewhere, + // check workflow stack for more details. + panicMsg := fmt.Sprintf("test timeout: %v, workflow stack: %v", + env.testTimeout, env.workflowDef.StackTrace()) + panic(panicMsg) + } + } + } + } +} + +func (env *testWorkflowEnvironmentImpl) shouldStopEventLoop() bool { + // Check if any detached children are still running if not disabled. + if !env.detachedChildWaitDisabled { + for _, handle := range env.runningWorkflows { + if env.workflowInfo.WorkflowExecution.ID == handle.env.workflowInfo.WorkflowExecution.ID { + // ignore root workflow + continue + } + + if !handle.handled && (handle.params.ParentClosePolicy == enumspb.PARENT_CLOSE_POLICY_ABANDON || + handle.params.ParentClosePolicy == enumspb.PARENT_CLOSE_POLICY_REQUEST_CANCEL) { + return false + } + } + } + + return env.isWorkflowCompleted +} + +func (env *testWorkflowEnvironmentImpl) registerDelayedCallback(f func(), delayDuration time.Duration) { + timerCallback := func(result *commonpb.Payloads, err error) { + f() + } + if delayDuration == 0 { + env.postCallback(f, false) + return + } + mainLoopCallback := func() { + env.newTimer(delayDuration, TimerOptions{}, timerCallback, false) + } + env.postCallback(mainLoopCallback, false) +} + +func (c *testCallbackHandle) processCallback() { + c.env.locker.Lock() + defer c.env.locker.Unlock() + c.callback() + if c.startWorkflowTask { + c.env.startWorkflowTask() + } +} + +func (env *testWorkflowEnvironmentImpl) autoFireNextTimer() bool { + if len(env.timers) == 0 { + return false + } + + // find next timer + var nextTimer *testTimerHandle + for _, t := range env.timers { + if nextTimer == nil { + nextTimer = t + } else if t.mockTimeToFire.Before(nextTimer.mockTimeToFire) || + (t.mockTimeToFire.Equal(nextTimer.mockTimeToFire) && t.timerID < nextTimer.timerID) { + nextTimer = t + } + } + + if nextTimer == nil { + return false + } + + // function to fire timer + fireTimer := func(th *testTimerHandle) { + skipDuration := th.mockTimeToFire.Sub(env.mockClock.Now()) + env.logger.Debug("Auto fire timer", + tagTimerID, th.timerID, + "TimerDuration", th.duration, + "TimeSkipped", skipDuration) + + // Move mockClock forward, this will fire the timer, and the timer callback will remove timer from timers. + env.mockClock.Add(skipDuration) + } + + // fire timer if there is no running activity + if env.runningCount == 0 { + if nextTimer.wallTimer != nil { + nextTimer.wallTimer.Stop() + nextTimer.wallTimer = nil + } + fireTimer(nextTimer) + return true + } + + durationToFire := nextTimer.mockTimeToFire.Sub(env.mockClock.Now()) + wallTimeToFire := env.wallClock.Now().Add(durationToFire) + + if nextTimer.wallTimer != nil && nextTimer.wallTimeToFire.Before(wallTimeToFire) { + // nextTimer already set, meaning we already have a wall clock timer for the nextTimer setup earlier. And the + // previously scheduled wall time to fire is before the wallTimeToFire calculated this time. This could happen + // if workflow was blocked while there was activity running, and when that activity completed, there are some + // other activities still running while the nextTimer is still that same nextTimer. In that case, we should not + // reset the wall time to fire for the nextTimer. + return false + } + if nextTimer.wallTimer != nil { + // wallTimer was scheduled, but the wall time to fire should be earlier based on current calculation. + nextTimer.wallTimer.Stop() + } + + // there is running activities, we would fire next timer only if wall time passed by nextTimer duration. + nextTimer.wallTimeToFire, nextTimer.wallTimer = wallTimeToFire, env.wallClock.AfterFunc(durationToFire, func() { + // make sure it is running in the main loop + nextTimer.env.postCallback(func() { + if timerHandle, ok := env.timers[getStringID(nextTimer.timerID)]; ok { + fireTimer(timerHandle) + } + }, true) + }) + + return false +} + +func (env *testWorkflowEnvironmentImpl) postCallback(cb func(), startWorkflowTask bool) { + env.callbackChannel <- testCallbackHandle{callback: cb, startWorkflowTask: startWorkflowTask, env: env} +} + +func (env *testWorkflowEnvironmentImpl) RequestCancelActivity(activityID ActivityID) { + handle, ok := env.getActivityHandle(activityID.id, env.workflowInfo.WorkflowExecution.RunID) + if !ok { + env.logger.Debug("RequestCancelActivity failed, Activity not exists or already completed.", tagActivityID, activityID) + return + } + activityInfo := env.getActivityInfo(activityID, handle.activityType) + env.logger.Debug("RequestCancelActivity", tagActivityID, activityID) + env.deleteHandle(activityID.id, env.workflowInfo.WorkflowExecution.RunID) + env.postCallback(func() { + handle.callback(nil, NewCanceledError()) + if env.onActivityCanceledListener != nil { + env.onActivityCanceledListener(activityInfo) + } + }, true) +} + +// RequestCancelTimer request to cancel timer on this testWorkflowEnvironmentImpl. +func (env *testWorkflowEnvironmentImpl) RequestCancelTimer(timerID TimerID) { + env.logger.Debug("RequestCancelTimer", tagTimerID, timerID) + timerHandle, ok := env.timers[timerID.id] + if !ok { + env.logger.Debug("RequestCancelTimer failed, TimerID not exists.", tagTimerID, timerID) + return + } + + delete(env.timers, timerID.id) + timerHandle.timer.Stop() + timerHandle.env.postCallback(func() { + timerHandle.callback(nil, NewCanceledError()) + if timerHandle.env.onTimerCanceledListener != nil { + timerHandle.env.onTimerCanceledListener(timerID.id) + } + }, true) +} + +func (env *testWorkflowEnvironmentImpl) Complete(result *commonpb.Payloads, err error) { + if env.isWorkflowCompleted { + env.logger.Debug("Workflow already completed.") + return + } + env.workflowDef.Close() + + dc := env.GetDataConverter() + env.isWorkflowCompleted = true + + if err != nil { + var continueAsNewErr *ContinueAsNewError + var canceledErr *CanceledError + var timeoutErr *TimeoutError + var workflowPanicErr *workflowPanicError + var workflowExecutionAlreadyStartedErr *serviceerror.WorkflowExecutionAlreadyStarted + var childWorkflowExecutionAlreadyStartedErr *ChildWorkflowExecutionAlreadyStartedError + if errors.As(err, &canceledErr) || errors.As(err, &continueAsNewErr) || errors.As(err, &timeoutErr) || errors.As(err, &workflowExecutionAlreadyStartedErr) || errors.As(err, &childWorkflowExecutionAlreadyStartedErr) { + env.testError = err + } else if errors.As(err, &workflowPanicErr) { + env.testError = newPanicError(workflowPanicErr.value, workflowPanicErr.stackTrace) + } else { + failure := env.failureConverter.ErrorToFailure(err) + env.testError = env.failureConverter.FailureToError(failure) + } + + if !env.isChildWorkflow() { + env.testError = NewWorkflowExecutionError( + env.WorkflowInfo().WorkflowExecution.ID, + env.WorkflowInfo().WorkflowExecution.RunID, + env.WorkflowInfo().WorkflowType.Name, + env.testError, + ) + } + } else { + env.testResult = newEncodedValue(result, dc) + } + + if env.isChildWorkflow() { + // this is completion of child workflow + childWorkflowID := env.workflowInfo.WorkflowExecution.ID + if childWorkflowHandle, ok := env.runningWorkflows[childWorkflowID]; ok && !childWorkflowHandle.handled { + // It is possible that child workflow could complete after cancellation. In that case, childWorkflowHandle + // would have already been removed from the runningWorkflows map by RequestCancelWorkflow(). + childWorkflowHandle.handled = true + // check if a retry is needed + if childWorkflowHandle.rerunAsChild() { + // rerun requested, so we don't want to post the error to parent workflow, return here. + return + } + + // no rerun, child workflow is done. + env.parentEnv.postCallback(func() { + // deliver result + if env.testError != nil { + childWorkflowHandle.err = NewChildWorkflowExecutionError( + defaultTestNamespace, + env.WorkflowInfo().WorkflowExecution.ID, + env.WorkflowInfo().WorkflowExecution.RunID, + env.WorkflowInfo().WorkflowType.Name, + 0, + 0, + enumspb.RETRY_STATE_UNSPECIFIED, + env.testError, + ) + } + childWorkflowHandle.callback(result, childWorkflowHandle.err) + if env.onChildWorkflowCompletedListener != nil { + env.onChildWorkflowCompletedListener(env.workflowInfo, env.testResult, childWorkflowHandle.err) + } + }, true /* true to trigger parent workflow to resume to handle child workflow's result */) + } + } + + // properly handle child workflows based on their ParentClosePolicy + env.handleParentClosePolicy() +} + +func (env *testWorkflowEnvironmentImpl) handleParentClosePolicy() { + for _, handle := range env.runningWorkflows { + if handle.env.parentEnv != nil && + env.workflowInfo.WorkflowExecution.ID == handle.env.parentEnv.workflowInfo.WorkflowExecution.ID { + + switch handle.params.ParentClosePolicy { + case enumspb.PARENT_CLOSE_POLICY_ABANDON: + // noop + case enumspb.PARENT_CLOSE_POLICY_TERMINATE: + handle.env.Complete(nil, newTerminatedError()) + case enumspb.PARENT_CLOSE_POLICY_REQUEST_CANCEL: + handle.env.cancelWorkflow(func(result *commonpb.Payloads, err error) {}) + } + } + } +} + +func (h *testWorkflowHandle) rerunAsChild() bool { + env := h.env + if !env.isChildWorkflow() { + return false + } + params := h.params + var continueAsNewErr *ContinueAsNewError + if errors.As(env.testError, &continueAsNewErr) { + params.Input = continueAsNewErr.Input + params.Header = continueAsNewErr.Header + params.RetryPolicy = convertToPBRetryPolicy(continueAsNewErr.RetryPolicy) + params.WorkflowType = continueAsNewErr.WorkflowType + params.TaskQueueName = continueAsNewErr.TaskQueueName + params.VersioningIntent = continueAsNewErr.VersioningIntent + params.WorkflowRunTimeout = continueAsNewErr.WorkflowRunTimeout + params.WorkflowTaskTimeout = continueAsNewErr.WorkflowTaskTimeout + // remove the current child workflow from the pending child workflow map because + // the childWorkflowID will be the same for retry run. + delete(env.runningWorkflows, env.workflowInfo.WorkflowExecution.ID) + env.parentEnv.ExecuteChildWorkflow(*params, h.callback, nil /* child workflow already started */) + return true + } + // pass down the last completion result + var result *commonpb.Payloads + // TODO (shtin): convert env.testResult to *commonpb.Payloads + if ev, ok := env.testResult.(*EncodedValue); ev != nil && ok { + result = ev.value + } + if result == nil { + // not successful run this time, carry over from whatever previous run pass to this run. + result = env.workflowInfo.lastCompletionResult + } + params.lastCompletionResult = result + if params.RetryPolicy != nil && env.testError != nil { + var expireTime time.Time + if params.WorkflowOptions.WorkflowExecutionTimeout > 0 { + expireTime = params.scheduledTime.Add(params.WorkflowOptions.WorkflowExecutionTimeout) + } + backoff := getRetryBackoffFromProtoRetryPolicy(params.RetryPolicy, env.workflowInfo.Attempt, env.testError, env.Now(), expireTime) + if backoff > 0 { + // remove the current child workflow from the pending child workflow map because + // the childWorkflowID will be the same for retry run. + delete(env.runningWorkflows, env.workflowInfo.WorkflowExecution.ID) + params.attempt++ + env.parentEnv.executeChildWorkflowWithDelay(backoff, *params, h.callback, nil /* child workflow already started */) + + return true + } + } + + if len(params.CronSchedule) > 0 { + schedule, err := cron.ParseStandard(params.CronSchedule) + if err != nil { + panic(fmt.Errorf("invalid cron schedule %v, err: %v", params.CronSchedule, err)) + } + + workflowNow := env.Now().In(time.UTC) + backoff := schedule.Next(workflowNow).Sub(workflowNow) + if backoff > 0 { + delete(env.runningWorkflows, env.workflowInfo.WorkflowExecution.ID) + params.attempt = 1 + params.scheduledTime = env.Now() + env.parentEnv.executeChildWorkflowWithDelay(backoff, *params, h.callback, nil /* child workflow already started */) + return true + } + } + + return false +} + +func (env *testWorkflowEnvironmentImpl) CompleteActivity(taskToken []byte, result interface{}, err error) error { + if taskToken == nil { + return errors.New("nil task token provided") + } + var data *commonpb.Payloads + if result != nil { + var encodeErr error + data, encodeErr = encodeArg(env.GetDataConverter(), result) + if encodeErr != nil { + return encodeErr + } + } + + activityID := ActivityID{id: string(taskToken)} + env.postCallback(func() { + activityHandle, ok := env.getActivityHandle(activityID.id, env.workflowInfo.WorkflowExecution.RunID) + if !ok { + env.logger.Debug("CompleteActivity: ActivityID not found, could be already completed or canceled.", + tagActivityID, activityID) + return + } + // We do allow canceled error to be passed here + cancelAllowed := true + request := convertActivityResultToRespondRequest("test-identity", taskToken, data, err, + env.GetDataConverter(), env.GetFailureConverter(), defaultTestNamespace, cancelAllowed, nil) + env.handleActivityResult(activityID, request, activityHandle.activityType, env.GetDataConverter()) + }, false /* do not auto schedule workflow task, because activity might be still pending */) + + return nil +} + +func (env *testWorkflowEnvironmentImpl) GetLogger() log.Logger { + return env.logger +} + +func (env *testWorkflowEnvironmentImpl) GetMetricsHandler() metrics.Handler { + return env.metricsHandler +} + +func (env *testWorkflowEnvironmentImpl) GetDataConverter() converter.DataConverter { + return env.dataConverter +} + +func (env *testWorkflowEnvironmentImpl) GetFailureConverter() converter.FailureConverter { + return env.failureConverter +} + +func (env *testWorkflowEnvironmentImpl) GetContextPropagators() []ContextPropagator { + return env.contextPropagators +} + +func (env *testWorkflowEnvironmentImpl) ExecuteActivity(parameters ExecuteActivityParams, callback ResultHandler) ActivityID { + ensureDefaultRetryPolicy(¶meters) + scheduleTaskAttr := &commandpb.ScheduleActivityTaskCommandAttributes{} + scheduleID := env.nextID() + if parameters.ActivityID == "" { + scheduleTaskAttr.ActivityId = getStringID(scheduleID) + } else { + scheduleTaskAttr.ActivityId = parameters.ActivityID + } + activityID := ActivityID{id: scheduleTaskAttr.GetActivityId()} + scheduleTaskAttr.ActivityType = &commonpb.ActivityType{Name: parameters.ActivityType.Name} + scheduleTaskAttr.TaskQueue = &taskqueuepb.TaskQueue{Name: parameters.TaskQueueName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} + scheduleTaskAttr.Input = parameters.Input + scheduleTaskAttr.ScheduleToCloseTimeout = durationpb.New(parameters.ScheduleToCloseTimeout) + scheduleTaskAttr.StartToCloseTimeout = durationpb.New(parameters.StartToCloseTimeout) + scheduleTaskAttr.ScheduleToStartTimeout = durationpb.New(parameters.ScheduleToStartTimeout) + scheduleTaskAttr.HeartbeatTimeout = durationpb.New(parameters.HeartbeatTimeout) + scheduleTaskAttr.RetryPolicy = parameters.RetryPolicy + scheduleTaskAttr.Header = parameters.Header + err := env.validateActivityScheduleAttributes(scheduleTaskAttr, env.WorkflowInfo().WorkflowRunTimeout) + if err != nil { + callback(nil, err) + return activityID + } + task := newTestActivityTask( + env.workflowInfo.WorkflowExecution.ID, + env.workflowInfo.WorkflowExecution.RunID, + env.workflowInfo.WorkflowType.Name, + env.workflowInfo.Namespace, + scheduleTaskAttr, + ) + + taskHandler := env.newTestActivityTaskHandler(parameters.TaskQueueName, parameters.DataConverter) + activityHandle := &testActivityHandle{callback: callback, activityType: parameters.ActivityType.Name} + + env.setActivityHandle(activityID.id, env.workflowInfo.WorkflowExecution.RunID, activityHandle) + env.runningCount++ + // activity runs in separate goroutinue outside of workflow dispatcher + // do callback in a defer to handle calls to runtime.Goexit inside the activity (which is done by t.FailNow) + go func() { + var result interface{} + defer func() { + panicErr := recover() + if result == nil && panicErr == nil { + failureErr := errors.New("activity called runtime.Goexit") + result = &workflowservice.RespondActivityTaskFailedRequest{ + Failure: env.failureConverter.ErrorToFailure(failureErr), + } + } else if panicErr != nil { + failureErr := newPanicError(fmt.Sprintf("%v", panicErr), "") + result = &workflowservice.RespondActivityTaskFailedRequest{ + Failure: env.failureConverter.ErrorToFailure(failureErr), + } + } + // post activity result to workflow dispatcher + env.postCallback(func() { + env.handleActivityResult(activityID, result, parameters.ActivityType.Name, parameters.DataConverter) + env.runningCount-- + }, false /* do not auto schedule workflow task, because activity might be still pending */) + }() + result = env.executeActivityWithRetryForTest(taskHandler, parameters, task) + }() + + return activityID +} + +func minDur(a *durationpb.Duration, b *durationpb.Duration) *durationpb.Duration { + ad, bd := a.AsDuration(), b.AsDuration() + if ad < bd { + return a + } + return b +} + +// Copy of the server function func (v *commandAttrValidator) validateActivityScheduleAttributes +func (env *testWorkflowEnvironmentImpl) validateActivityScheduleAttributes( + attributes *commandpb.ScheduleActivityTaskCommandAttributes, + runTimeout time.Duration, +) error { + + if attributes == nil { + return serviceerror.NewInvalidArgument("ScheduleActivityTaskCommandAttributes is not set on command.") + } + + defaultTaskQueueName := "" + if _, err := env.validatedTaskQueue(attributes.TaskQueue, defaultTaskQueueName); err != nil { + return err + } + + if attributes.GetActivityId() == "" { + return serviceerror.NewInvalidArgument("ActivityId is not set on command.") + } + + if attributes.ActivityType == nil || attributes.ActivityType.GetName() == "" { + return serviceerror.NewInvalidArgument("ActivityType is not set on command.") + } + + if err := env.validateRetryPolicy(attributes.RetryPolicy); err != nil { + return err + } + + if len(attributes.GetActivityId()) > maxIDLengthLimit { + return serviceerror.NewInvalidArgument("ActivityID exceeds length limit.") + } + + if len(attributes.GetActivityType().GetName()) > maxIDLengthLimit { + return serviceerror.NewInvalidArgument("ActivityType exceeds length limit.") + } + + // Only attempt to deduce and fill in unspecified timeouts only when all timeouts are non-negative. + if attributes.GetScheduleToCloseTimeout().AsDuration() < 0 || attributes.GetScheduleToStartTimeout().AsDuration() < 0 || + attributes.GetStartToCloseTimeout().AsDuration() < 0 || attributes.GetHeartbeatTimeout().AsDuration() < 0 { + return serviceerror.NewInvalidArgument("A valid timeout may not be negative.") + } + + validScheduleToClose := attributes.GetScheduleToCloseTimeout().AsDuration() > 0 + validScheduleToStart := attributes.GetScheduleToStartTimeout().AsDuration() > 0 + validStartToClose := attributes.GetStartToCloseTimeout().AsDuration() > 0 + + if validScheduleToClose { + if validScheduleToStart { + attributes.ScheduleToStartTimeout = minDur(attributes.GetScheduleToStartTimeout(), attributes.GetScheduleToCloseTimeout()) + } else { + attributes.ScheduleToStartTimeout = attributes.GetScheduleToCloseTimeout() + } + if validStartToClose { + attributes.StartToCloseTimeout = minDur(attributes.GetStartToCloseTimeout(), attributes.GetScheduleToCloseTimeout()) + } else { + attributes.StartToCloseTimeout = attributes.GetScheduleToCloseTimeout() + } + } else if validStartToClose { + // We are in !validScheduleToClose due to the first if above + attributes.ScheduleToCloseTimeout = durationpb.New(runTimeout) + if !validScheduleToStart { + attributes.ScheduleToStartTimeout = durationpb.New(runTimeout) + } + } else { + // Deduction failed as there's not enough information to fill in missing timeouts. + return serviceerror.NewInvalidArgument("A valid StartToClose or ScheduleToCloseTimeout is not set on command.") + } + // ensure activity timeout never larger than workflow timeout + if runTimeout > 0 { + if attributes.GetScheduleToCloseTimeout().AsDuration() > runTimeout { + attributes.ScheduleToCloseTimeout = durationpb.New(runTimeout) + } + if attributes.GetScheduleToStartTimeout().AsDuration() > runTimeout { + attributes.ScheduleToStartTimeout = durationpb.New(runTimeout) + } + if attributes.GetStartToCloseTimeout().AsDuration() > runTimeout { + attributes.StartToCloseTimeout = durationpb.New(runTimeout) + } + if attributes.GetHeartbeatTimeout().AsDuration() > runTimeout { + attributes.HeartbeatTimeout = durationpb.New(runTimeout) + } + } + attributes.HeartbeatTimeout = minDur(attributes.GetHeartbeatTimeout(), attributes.GetScheduleToCloseTimeout()) + return nil +} + +// Copy of the service func (v *commandAttrValidator) validatedTaskQueue +func (env *testWorkflowEnvironmentImpl) validatedTaskQueue( + taskQueue *taskqueuepb.TaskQueue, + defaultVal string, +) (*taskqueuepb.TaskQueue, error) { + + if taskQueue == nil { + taskQueue = &taskqueuepb.TaskQueue{Kind: enumspb.TASK_QUEUE_KIND_NORMAL} + } + + if taskQueue.GetName() == "" { + if defaultVal == "" { + return taskQueue, serviceerror.NewInvalidArgument("missing task queue name") + } + taskQueue.Name = defaultVal + return taskQueue, nil + } + + name := taskQueue.GetName() + if len(name) > maxIDLengthLimit { + return taskQueue, serviceerror.NewInvalidArgument(fmt.Sprintf("task queue name exceeds length limit of %v", maxIDLengthLimit)) + } + + if strings.HasPrefix(name, reservedTaskQueuePrefix) { + return taskQueue, serviceerror.NewInvalidArgument(fmt.Sprintf("task queue name cannot start with reserved prefix %v", reservedTaskQueuePrefix)) + } + + return taskQueue, nil +} + +// copy of the service func ValidateRetryPolicy(policy *commonpb.RetryPolicy) +func (env *testWorkflowEnvironmentImpl) validateRetryPolicy(policy *commonpb.RetryPolicy) error { + if policy == nil { + // nil policy is valid which means no retry + return nil + } + + if policy.GetMaximumAttempts() == 1 { + // One maximum attempt effectively disable retries. Validating the + // rest of the arguments is pointless + return nil + } + if policy.GetInitialInterval().AsDuration() < 0 { + return serviceerror.NewInvalidArgument("InitialInterval cannot be negative on retry policy.") + } + if policy.GetBackoffCoefficient() < 1 { + return serviceerror.NewInvalidArgument("BackoffCoefficient cannot be less than 1 on retry policy.") + } + if policy.GetMaximumInterval().AsDuration() < 0 { + return serviceerror.NewInvalidArgument("MaximumInterval cannot be negative on retry policy.") + } + if policy.GetMaximumInterval().AsDuration() > 0 && policy.GetMaximumInterval().AsDuration() < policy.GetInitialInterval().AsDuration() { + return serviceerror.NewInvalidArgument("MaximumInterval cannot be less than InitialInterval on retry policy.") + } + if policy.GetMaximumAttempts() < 0 { + return serviceerror.NewInvalidArgument("MaximumAttempts cannot be negative on retry policy.") + } + return nil +} + +func (env *testWorkflowEnvironmentImpl) getActivityHandle(activityID, runID string) (*testActivityHandle, bool) { + handle, ok := env.activities[env.makeUniqueActivityID(activityID, runID)] + return handle, ok +} + +func (env *testWorkflowEnvironmentImpl) setActivityHandle(activityID, runID string, handle *testActivityHandle) { + env.activities[env.makeUniqueActivityID(activityID, runID)] = handle +} + +func (env *testWorkflowEnvironmentImpl) deleteHandle(activityID, runID string) { + delete(env.activities, env.makeUniqueActivityID(activityID, runID)) +} + +func (env *testWorkflowEnvironmentImpl) makeUniqueActivityID(activityID, runID string) string { + // ActivityID is unique per workflow, but different workflow could have same activityID. + // Make the key unique globally as we share the same collection for all running workflows in test. + return fmt.Sprintf("%v_%v", runID, activityID) +} + +func (env *testWorkflowEnvironmentImpl) executeActivityWithRetryForTest( + taskHandler ActivityTaskHandler, + parameters ExecuteActivityParams, + task *workflowservice.PollActivityTaskQueueResponse, +) (result interface{}) { + var expireTime time.Time + if parameters.ScheduleToCloseTimeout > 0 { + expireTime = env.Now().Add(parameters.ScheduleToCloseTimeout) + } + + for { + var err error + result, err = taskHandler.Execute(parameters.TaskQueueName, task) + if err != nil { + if err == context.DeadlineExceeded { + return err + } + panic(err) + } + + // check if a retry is needed + if request, ok := result.(*workflowservice.RespondActivityTaskFailedRequest); ok && parameters.RetryPolicy != nil { + failure := request.GetFailure() + + if failure.GetApplicationFailureInfo().GetNonRetryable() { + break + } + + p := fromProtoRetryPolicy(parameters.RetryPolicy) + backoff := getRetryBackoffWithNowTime(p, task.GetAttempt(), env.failureConverter.FailureToError(failure), env.Now(), expireTime) + if backoff > 0 { + // need a retry + waitCh := make(chan struct{}) + + // register the delayed call back first, otherwise other timers may be fired before the retry timer + // is enqueued. + env.registerDelayedCallback(func() { + env.runningCount++ + task.Attempt = task.GetAttempt() + 1 + activityID := ActivityID{id: string(task.TaskToken)} + if ah, ok := env.getActivityHandle(activityID.id, task.WorkflowExecution.RunId); ok { + task.HeartbeatDetails = ah.heartbeatDetails + } + close(waitCh) + }, backoff) + env.postCallback(func() { env.runningCount-- }, false) + + <-waitCh + continue + } + } + + // no retry + break + } + + return +} + +func fromProtoRetryPolicy(p *commonpb.RetryPolicy) *RetryPolicy { + return &RetryPolicy{ + InitialInterval: p.GetInitialInterval().AsDuration(), + BackoffCoefficient: p.GetBackoffCoefficient(), + MaximumInterval: p.GetMaximumInterval().AsDuration(), + MaximumAttempts: p.GetMaximumAttempts(), + NonRetryableErrorTypes: p.NonRetryableErrorTypes, + } +} + +func getRetryBackoffFromProtoRetryPolicy(prp *commonpb.RetryPolicy, attempt int32, err error, now, expireTime time.Time) time.Duration { + if prp == nil { + return noRetryBackoff + } + + p := fromProtoRetryPolicy(prp) + return getRetryBackoffWithNowTime(p, attempt, err, now, expireTime) +} + +func ensureDefaultRetryPolicy(parameters *ExecuteActivityParams) { + // ensure default retry policy + if parameters.RetryPolicy == nil { + parameters.RetryPolicy = &commonpb.RetryPolicy{} + } + + if parameters.RetryPolicy.InitialInterval == nil || parameters.RetryPolicy.InitialInterval.AsDuration() == 0 { + parameters.RetryPolicy.InitialInterval = durationpb.New(time.Second) + } + if parameters.RetryPolicy.MaximumInterval == nil || parameters.RetryPolicy.MaximumInterval.AsDuration() == 0 { + parameters.RetryPolicy.MaximumInterval = parameters.RetryPolicy.InitialInterval + } + if parameters.RetryPolicy.BackoffCoefficient == 0 { + parameters.RetryPolicy.BackoffCoefficient = 2 + } + + // NOTE: the default MaximumAttempts for retry policy set by server is 0 which means unlimited retries. + // However, unlimited retry with automatic fast forward clock in test framework will cause the CPU to spin and test + // to go forever. So we need to set a reasonable default max attempts for unit test. + if parameters.RetryPolicy.MaximumAttempts == 0 { + parameters.RetryPolicy.MaximumAttempts = defaultMaximumAttemptsForUnitTest + } +} + +func (env *testWorkflowEnvironmentImpl) ExecuteLocalActivity(params ExecuteLocalActivityParams, callback LocalActivityResultHandler) LocalActivityID { + activityID := getStringID(env.nextID()) + ae := &activityExecutor{name: getActivityFunctionName(env.registry, params.ActivityFn), fn: params.ActivityFn} + if at, _ := getValidatedActivityFunction(params.ActivityFn, params.InputArgs, env.registry); at != nil { + // local activity could be registered, if so use the registered name. This name is only used to find a mock. + ae.name = at.Name + } + // We have to skip the interceptors on the first call because + // ExecuteWithActualArgs is actually invoked twice to support a mock activity + // function result + ae.skipInterceptors = true + aew := &activityExecutorWrapper{activityExecutor: ae, env: env} + + // substitute the local activity function so we could replace with mock if it is supplied. + params.ActivityFn = func(ctx context.Context, inputArgs ...interface{}) (*commonpb.Payloads, error) { + return aew.ExecuteWithActualArgs(ctx, params.InputArgs) + } + + task := newLocalActivityTask(params, callback, activityID) + taskHandler := localActivityTaskHandler{ + userContext: env.workerOptions.BackgroundActivityContext, + metricsHandler: env.metricsHandler, + logger: env.logger, + dataConverter: env.dataConverter, + contextPropagators: env.contextPropagators, + interceptors: env.registry.interceptors, + } + + env.localActivities[activityID] = task + env.runningCount++ + + go func() { + result := taskHandler.executeLocalActivityTask(task) + env.postCallback(func() { + env.handleLocalActivityResult(result) + env.runningCount-- + }, false) + }() + + return LocalActivityID{id: activityID} +} + +func (env *testWorkflowEnvironmentImpl) RequestCancelLocalActivity(activityID LocalActivityID) { + task, ok := env.localActivities[activityID.id] + if !ok { + env.logger.Debug("RequestCancelLocalActivity failed, LocalActivity not exists or already completed.", tagActivityID, activityID) + return + } + env.logger.Debug("RequestCancelLocalActivity", tagActivityID, activityID) + task.cancel() +} + +func (env *testWorkflowEnvironmentImpl) handleActivityResult(activityID ActivityID, result interface{}, activityType string, + dataConverter converter.DataConverter) { + env.logger.Debug(fmt.Sprintf("handleActivityResult: %T.", result), + tagActivityID, activityID, tagActivityType, activityType) + activityInfo := env.getActivityInfo(activityID, activityType) + if result == ErrActivityResultPending { + // In case activity returns ErrActivityResultPending, the respond will be nil, and we don't need to do anything. + // Activity will need to complete asynchronously using CompleteActivity(). + if env.onActivityCompletedListener != nil { + env.onActivityCompletedListener(activityInfo, nil, ErrActivityResultPending) + } + return + } + + // this is running in dispatcher + activityHandle, ok := env.getActivityHandle(activityID.id, activityInfo.WorkflowExecution.RunID) + if !ok { + env.logger.Debug("handleActivityResult: ActivityID not exists, could be already completed or canceled.", + tagActivityID, activityID) + return + } + + env.deleteHandle(activityID.id, activityInfo.WorkflowExecution.RunID) + + var blob *commonpb.Payloads + var err error + + switch request := result.(type) { + case *workflowservice.RespondActivityTaskCanceledRequest: + details := newEncodedValues(request.Details, dataConverter) + err = env.wrapActivityError( + activityID, + activityType, + enumspb.RETRY_STATE_NON_RETRYABLE_FAILURE, + NewCanceledError(details), + ) + activityHandle.callback(nil, err) + case *workflowservice.RespondActivityTaskFailedRequest: + err = env.wrapActivityError( + activityID, + activityType, + enumspb.RETRY_STATE_UNSPECIFIED, + env.failureConverter.FailureToError(request.GetFailure()), + ) + activityHandle.callback(nil, err) + case *workflowservice.RespondActivityTaskCompletedRequest: + blob = request.Result + activityHandle.callback(blob, nil) + default: + if result == context.DeadlineExceeded { + err = env.wrapActivityError( + activityID, + activityType, + enumspb.RETRY_STATE_TIMEOUT, + NewTimeoutError("Activity timeout", enumspb.TIMEOUT_TYPE_START_TO_CLOSE, context.DeadlineExceeded), + ) + activityHandle.callback(nil, err) + } else { + panic(fmt.Sprintf("unsupported respond type %T", result)) + } + } + + if env.onActivityCompletedListener != nil { + if err != nil { + env.onActivityCompletedListener(activityInfo, nil, err) + } else { + env.onActivityCompletedListener(activityInfo, newEncodedValue(blob, dataConverter), nil) + } + } + + env.startWorkflowTask() +} + +func (env *testWorkflowEnvironmentImpl) wrapActivityError(activityID ActivityID, activityType string, retryState enumspb.RetryState, activityErr error) error { + if activityErr == nil { + return nil + } + + return NewActivityError( + 0, + 0, + env.identity, + &commonpb.ActivityType{Name: activityType}, + activityID.id, + retryState, + activityErr, + ) +} + +func (env *testWorkflowEnvironmentImpl) handleLocalActivityResult(result *localActivityResult) { + activityID := ActivityID{id: result.task.activityID} + activityType := getActivityFunctionName(env.registry, result.task.params.ActivityFn) + env.logger.Debug(fmt.Sprintf("handleLocalActivityResult: Err: %v, Result: %v.", result.err, result.result), + tagActivityID, activityID, tagActivityType, activityType) + + activityInfo := env.getActivityInfo(activityID, activityType) + task, ok := env.localActivities[activityID.id] + if !ok { + env.logger.Debug("handleLocalActivityResult: ActivityID not exists, could be already completed or canceled.", + tagActivityID, activityID) + return + } + delete(env.localActivities, activityID.id) + // If error is present do not return value + if result.err != nil && result.result != nil { + result.result = nil + } + // Always return CanceledError for canceled tasks + if task.canceled { + var canceledErr *CanceledError + if !errors.As(result.err, &canceledErr) { + result.err = NewCanceledError() + result.result = nil + } + } + lar := &LocalActivityResultWrapper{ + Err: env.wrapActivityError(activityID, activityType, enumspb.RETRY_STATE_UNSPECIFIED, result.err), + Result: result.result, + Backoff: noRetryBackoff, + Attempt: 1, + } + if result.task.retryPolicy != nil && result.err != nil { + lar.Backoff = getRetryBackoff(result, env.Now()) + lar.Attempt = task.attempt + } + task.callback(lar) + var canceledErr *CanceledError + if errors.As(lar.Err, &canceledErr) { + if env.onLocalActivityCanceledListener != nil { + env.onLocalActivityCanceledListener(activityInfo) + } + } else if env.onLocalActivityCompletedListener != nil { + env.onLocalActivityCompletedListener(activityInfo, newEncodedValue(result.result, env.GetDataConverter()), nil) + } + env.startWorkflowTask() +} + +// runBeforeMockCallReturns is registered as mock call's RunFn by *mock.Call.Run(fn). It will be called by testify's +// mock.MethodCalled() before it returns. +func (env *testWorkflowEnvironmentImpl) runBeforeMockCallReturns(call *MockCallWrapper, args mock.Arguments) { + var waitDuration time.Duration + if call.waitDuration != nil { + waitDuration = call.waitDuration() + } + if waitDuration > 0 { + // we want this mock call to block until the wait duration is elapsed (on workflow clock). + waitCh := make(chan time.Time) + env.registerDelayedCallback(func() { + env.runningCount++ // increase runningCount as the mock call is ready to resume. + waitCh <- env.Now() // this will unblock mock call + }, waitDuration) + + // make sure decrease runningCount after delayed callback is posted + env.postCallback(func() { + env.runningCount-- // reduce runningCount, since this mock call is about to be blocked. + }, false) + <-waitCh // this will block until mock clock move forward by waitDuration + } + + // run the actual runFn if it was setup + if call.runFn != nil { + call.runFn(args) + } +} + +// Execute executes the activity code. +func (a *activityExecutorWrapper) Execute(ctx context.Context, input *commonpb.Payloads) (*commonpb.Payloads, error) { + activityInfo := GetActivityInfo(ctx) + // If the activity was cancelled before it starts here, we do not execute and + // instead return cancelled + a.env.locker.Lock() + _, handleExists := a.env.getActivityHandle(activityInfo.ActivityID, activityInfo.WorkflowExecution.RunID) + a.env.locker.Unlock() + if !handleExists { + return nil, NewCanceledError() + } + + dc := getDataConverterFromActivityCtx(ctx) + if a.env.onActivityStartedListener != nil { + waitCh := make(chan struct{}) + a.env.postCallback(func() { + a.env.onActivityStartedListener(&activityInfo, ctx, newEncodedValues(input, dc)) + close(waitCh) + }, false) + <-waitCh // wait until listener returns + } + + m := &mockWrapper{env: a.env, name: a.name, fn: a.fn, isWorkflow: false, dataConverter: dc} + if mockRet := m.getActivityMockReturn(ctx, input); mockRet != nil { + return m.executeMock(ctx, input, mockRet) + } + + return a.activityExecutor.Execute(ctx, input) +} + +// ExecuteWithActualArgs executes the activity code. +func (a *activityExecutorWrapper) ExecuteWithActualArgs(ctx context.Context, inputArgs []interface{}) (*commonpb.Payloads, error) { + activityInfo := GetActivityInfo(ctx) + if a.env.onLocalActivityStartedListener != nil { + waitCh := make(chan struct{}) + a.env.postCallback(func() { + a.env.onLocalActivityStartedListener(&activityInfo, ctx, inputArgs) + close(waitCh) + }, false) + <-waitCh + } + + m := &mockWrapper{env: a.env, name: a.name, fn: a.fn, isWorkflow: false} + if mockRet := m.getActivityMockReturnWithActualArgs(ctx, inputArgs); mockRet != nil { + // check if mock returns function which must match to the actual function. + if mockFn := m.getMockFn(mockRet); mockFn != nil { + executor := &activityExecutor{name: m.name, fn: mockFn} + return executor.ExecuteWithActualArgs(ctx, inputArgs) + } + return m.getMockValue(mockRet) + } + + return a.activityExecutor.ExecuteWithActualArgs(ctx, inputArgs) +} + +// Execute executes the workflow code. +func (w *workflowExecutorWrapper) Execute(ctx Context, input *commonpb.Payloads) (result *commonpb.Payloads, err error) { + env := w.env + if env.isChildWorkflow() && env.onChildWorkflowStartedListener != nil { + env.onChildWorkflowStartedListener(GetWorkflowInfo(ctx), ctx, newEncodedValues(input, w.env.GetDataConverter())) + } + + if !env.isChildWorkflow() { + // This is to prevent auto-forwarding mock clock before main workflow starts. For child workflow, we increase + // the counter in env.ExecuteChildWorkflow(). We cannot do it here for child workflow, because we need to make + // sure the counter is increased before returning from ExecuteChildWorkflow(). + env.runningCount++ + } + + m := &mockWrapper{env: env, name: w.workflowType, fn: w.fn, isWorkflow: true, dataConverter: env.GetDataConverter()} + // This method is called by workflow's dispatcher. In this test suite, it is run in the main loop. We cannot block + // the main loop, but the mock could block if it is configured to wait. So we need to use a separate goroutinue to + // run the mock, and resume after mock call returns. + mockReadyChannel := NewChannel(ctx) + // make a copy of the context for getWorkflowMockReturn() call to avoid race condition + _, ctxCopy, err := newWorkflowContext(w.env, nil) + if err != nil { + return nil, err + } + go func() { + // getWorkflowMockReturn could block if mock is configured to wait. The returned mockRet is what has been configured + // for the mock by using MockCallWrapper.Return(). The mockRet could be mock values or mock function. We process + // the returned mockRet by calling executeMock() later in the main thread after it is send over via mockReadyChannel. + mockRet := m.getWorkflowMockReturn(ctxCopy, input) + env.postCallback(func() { + mockReadyChannel.SendAsync(mockRet) + }, true /* true to trigger the dispatcher for this workflow so it resume from mockReadyChannel block*/) + }() + + var mockRet mock.Arguments + // This will block workflow dispatcher (on temporal channel), which the dispatcher understand and will return from + // ExecuteUntilAllBlocked() so the main loop is not blocked. The dispatcher will unblock when getWorkflowMockReturn() returns. + mockReadyChannel.Receive(ctx, &mockRet) + + // reduce runningCount to allow auto-forwarding mock clock after current workflow dispatcher run is blocked (aka + // ExecuteUntilAllBlocked() returns). + env.runningCount-- + w.env.workflowFunctionExecuting = true + + childWE := env.workflowInfo.WorkflowExecution + var startedErr error + if mockRet != nil { + // workflow was mocked. + result, err = m.executeMock(ctx, input, mockRet) + if env.isChildWorkflow() && err == ErrMockStartChildWorkflowFailed { + childWE, startedErr = WorkflowExecution{}, err + } + } + + if env.isChildWorkflow() && env.startedHandler != nil /* startedHandler could be nil for retry */ { + // notify parent that child workflow is started + env.parentEnv.postCallback(func() { + env.startedHandler(childWE, startedErr) + }, true) + } + + if mockRet != nil { + return result, err + } + + // no mock, so call the actual workflow + return w.workflowExecutor.Execute(ctx, input) +} + +func (m *mockWrapper) getCtxArg(ctx interface{}) []interface{} { + fnType := reflect.TypeOf(m.fn) + if fnType.NumIn() > 0 { + if (!m.isWorkflow && isActivityContext(fnType.In(0))) || + (m.isWorkflow && isWorkflowContext(fnType.In(0))) { + return []interface{}{ctx} + } + } + return nil +} + +func (m *mockWrapper) getActivityMockReturn(ctx interface{}, input *commonpb.Payloads) (retArgs mock.Arguments) { + if _, ok := m.env.expectedActivityMockCalls[m.name]; !ok { + // no mock + return nil + } + + return m.getMockReturn(ctx, input, m.env.activityMock) +} + +func (m *mockWrapper) getWorkflowMockReturn(ctx interface{}, input *commonpb.Payloads) (retArgs mock.Arguments) { + if _, ok := m.env.expectedWorkflowMockCalls[m.name]; !ok { + // no mock + return nil + } + + return m.getMockReturn(ctx, input, m.env.workflowMock) +} + +func (m *mockWrapper) getMockReturn(ctx interface{}, input *commonpb.Payloads, envMock *mock.Mock) (retArgs mock.Arguments) { + fnType := reflect.TypeOf(m.fn) + reflectArgs, err := decodeArgs(m.dataConverter, fnType, input) + if err != nil { + panic(fmt.Sprintf("Decode error: %v in %v of type %T", err.Error(), m.name, m.fn)) + } + realArgs := m.getCtxArg(ctx) + for _, arg := range reflectArgs { + realArgs = append(realArgs, arg.Interface()) + } + + return envMock.MethodCalled(m.name, realArgs...) +} + +func (m *mockWrapper) getActivityMockReturnWithActualArgs(ctx interface{}, inputArgs []interface{}) (retArgs mock.Arguments) { + if _, ok := m.env.expectedActivityMockCalls[m.name]; !ok { + // no mock + return nil + } + + return m.getMockReturnWithActualArgs(ctx, inputArgs, m.env.activityMock) +} + +func (m *mockWrapper) getMockReturnWithActualArgs(ctx interface{}, inputArgs []interface{}, envMock *mock.Mock) (retArgs mock.Arguments) { + realArgs := m.getCtxArg(ctx) + realArgs = append(realArgs, inputArgs...) + return envMock.MethodCalled(m.name, realArgs...) +} + +func (m *mockWrapper) getMockFn(mockRet mock.Arguments) interface{} { + fnName := m.name + mockRetLen := len(mockRet) + if mockRetLen == 0 { + panic(fmt.Sprintf("mock of %v has no returns", fnName)) + } + + fnType := reflect.TypeOf(m.fn) + // check if mock returns function which must match to the actual function. + mockFn := mockRet.Get(0) + mockFnType := reflect.TypeOf(mockFn) + if mockFnType != nil && mockFnType.Kind() == reflect.Func { + if mockFnType != fnType { + fnName, _ := getFunctionName(m.fn) + // mockDummyActivity is used to register mocks by name + if fnName != "mockDummyActivity" { + panic(fmt.Sprintf("mock of %v has incorrect return function, expected %v, but actual is %v", + fnName, fnType, mockFnType)) + } + } + return mockFn + } + return nil +} + +func (m *mockWrapper) getMockValue(mockRet mock.Arguments) (*commonpb.Payloads, error) { + fnName := m.name + mockRetLen := len(mockRet) + fnType := reflect.TypeOf(m.fn) + // check if mockRet have same types as function's return types + if mockRetLen != fnType.NumOut() { + panic(fmt.Sprintf("mock of %v has incorrect number of returns, expected %d, but actual is %d", + fnName, fnType.NumOut(), mockRetLen)) + } + // we already verified function either has 1 return value (error) or 2 return values (result, error) + var retErr error + mockErr := mockRet[mockRetLen-1] // last mock return must be error + if mockErr == nil { + retErr = nil + } else if err, ok := mockErr.(error); ok { + retErr = err + } else { + panic(fmt.Sprintf("mock of %v has incorrect return type, expected error, but actual is %T (%v)", + fnName, mockErr, mockErr)) + } + + switch mockRetLen { + case 1: + return nil, retErr + case 2: + expectedType := fnType.Out(0) + mockResult := mockRet[0] + if mockResult == nil { + switch expectedType.Kind() { + case reflect.Ptr, reflect.Interface, reflect.Map, reflect.Slice, reflect.Array: + // these are supported nil-able types. (reflect.Chan, reflect.Func are nil-able, but not supported) + return nil, retErr + default: + panic(fmt.Sprintf("mock of %v has incorrect return type, expected %v, but actual is %T (%v)", + fnName, expectedType, mockResult, mockResult)) + } + } else { + if !reflect.TypeOf(mockResult).AssignableTo(expectedType) { + panic(fmt.Sprintf("mock of %v has incorrect return type, expected %v, but actual is %T (%v)", + fnName, expectedType, mockResult, mockResult)) + } + result, encodeErr := encodeArg(m.env.GetDataConverter(), mockResult) + if encodeErr != nil { + panic(fmt.Sprintf("encode result from mock of %v failed: %v", fnName, encodeErr)) + } + return result, retErr + } + default: + // this will never happen, panic just in case + panic("mock should either have 1 return value (error) or 2 return values (result, error)") + } +} + +func (m *mockWrapper) executeMock(ctx interface{}, input *commonpb.Payloads, mockRet mock.Arguments) (result *commonpb.Payloads, err error) { + // have to handle panics here to support calling ExecuteChildWorkflow(...).GetChildWorkflowExecution().Get(...) + // when a child is mocked. + defer func() { + if r := recover(); r != nil { + st := getStackTrace("executeMock", "panic", 4) + err = newPanicError(r, st) + } + }() + + fnName := m.name + // check if mock returns function which must match to the actual function. + if mockFn := m.getMockFn(mockRet); mockFn != nil { + // we found a mock function that matches to actual function, so call that mockFn + if m.isWorkflow { + executor := &workflowExecutor{workflowType: fnName, fn: mockFn} + return executor.Execute(ctx.(Context), input) + } + executor := &activityExecutor{name: fnName, fn: mockFn} + return executor.Execute(ctx.(context.Context), input) + } + + return m.getMockValue(mockRet) +} + +func (env *testWorkflowEnvironmentImpl) newTestActivityTaskHandler(taskQueue string, dataConverter converter.DataConverter) ActivityTaskHandler { + setWorkerOptionsDefaults(&env.workerOptions) + params := workerExecutionParameters{ + TaskQueue: taskQueue, + Identity: env.identity, + MetricsHandler: env.metricsHandler, + Logger: env.logger, + UserContext: env.workerOptions.BackgroundActivityContext, + FailureConverter: env.failureConverter, + DataConverter: dataConverter, + WorkerStopChannel: env.workerStopChannel, + ContextPropagators: env.contextPropagators, + } + ensureRequiredParams(¶ms) + if params.UserContext == nil { + params.UserContext = context.Background() + } + if env.workerOptions.EnableSessionWorker && env.sessionEnvironment == nil { + env.sessionEnvironment = newTestSessionEnvironment(env, ¶ms, env.workerOptions.MaxConcurrentSessionExecutionSize) + } + params.UserContext = context.WithValue(params.UserContext, sessionEnvironmentContextKey, env.sessionEnvironment) + registry := env.registry + if len(registry.getRegisteredActivities()) == 0 { + panic(fmt.Sprintf("no activity is registered for taskqueue '%v'", taskQueue)) + } + + getActivity := func(name string) activity { + tlsa, ok := env.taskQueueSpecificActivities[name] + if ok { + _, ok := tlsa.taskQueues[taskQueue] + if !ok { + // activity are bind to specific task queue but not to current task queue + return nil + } + } + + activity, ok := registry.GetActivity(name) + if !ok { + return nil + } + ae := &activityExecutor{name: activity.ActivityType().Name, fn: activity.GetFunction()} + + if env.sessionEnvironment != nil { + // Special handling for session creation and completion activities. + // If real creation activity is used, it will block timers from autofiring. + if ae.name == sessionCreationActivityName { + ae.fn = sessionCreationActivityForTest + } + if ae.name == sessionCompletionActivityName { + ae.fn = sessionCompletionActivityForTest + } + } + return &activityExecutorWrapper{activityExecutor: ae, env: env} + } + + taskHandler := newActivityTaskHandlerWithCustomProvider(env.service, params, registry, getActivity) + return taskHandler +} + +func newTestActivityTask(workflowID, runID, workflowTypeName, namespace string, + attr *commandpb.ScheduleActivityTaskCommandAttributes) *workflowservice.PollActivityTaskQueueResponse { + activityID := attr.GetActivityId() + now := time.Now() + task := &workflowservice.PollActivityTaskQueueResponse{ + Attempt: 1, + WorkflowExecution: &commonpb.WorkflowExecution{ + WorkflowId: workflowID, + RunId: runID, + }, + ActivityId: activityID, + TaskToken: []byte(activityID), // use activityID as TaskToken so we can map TaskToken in heartbeat calls. + ActivityType: &commonpb.ActivityType{Name: attr.GetActivityType().GetName()}, + Input: attr.GetInput(), + ScheduledTime: timestamppb.New(now), + ScheduleToCloseTimeout: attr.GetScheduleToCloseTimeout(), + StartedTime: timestamppb.New(now), + StartToCloseTimeout: attr.GetStartToCloseTimeout(), + HeartbeatTimeout: attr.GetHeartbeatTimeout(), + WorkflowType: &commonpb.WorkflowType{ + Name: workflowTypeName, + }, + WorkflowNamespace: namespace, + Header: attr.GetHeader(), + } + return task +} + +func (env *testWorkflowEnvironmentImpl) newTimer( + d time.Duration, + options TimerOptions, + callback ResultHandler, + notifyListener bool, +) *TimerID { + nextID := env.nextID() + timerInfo := &TimerID{id: getStringID(nextID)} + timer := env.mockClock.AfterFunc(d, func() { + delete(env.timers, timerInfo.id) + env.postCallback(func() { + callback(nil, nil) + if notifyListener && env.onTimerFiredListener != nil { + env.onTimerFiredListener(timerInfo.id) + } + }, true) + }) + env.timers[timerInfo.id] = &testTimerHandle{ + env: env, + callback: callback, + timer: timer, + mockTimeToFire: env.mockClock.Now().Add(d), + wallTimeToFire: env.wallClock.Now().Add(d), + duration: d, + timerID: nextID, + } + if notifyListener && env.onTimerScheduledListener != nil { + env.onTimerScheduledListener(timerInfo.id, d) + } + return timerInfo +} + +func (env *testWorkflowEnvironmentImpl) NewTimer( + d time.Duration, + options TimerOptions, + callback ResultHandler, +) *TimerID { + return env.newTimer(d, options, callback, true) +} + +func (env *testWorkflowEnvironmentImpl) Now() time.Time { + return env.mockClock.Now() +} + +func (env *testWorkflowEnvironmentImpl) WorkflowInfo() *WorkflowInfo { + return env.workflowInfo +} + +func (env *testWorkflowEnvironmentImpl) TypedSearchAttributes() SearchAttributes { + return convertToTypedSearchAttributes(env.logger, env.workflowInfo.SearchAttributes.GetIndexedFields()) +} + +func (env *testWorkflowEnvironmentImpl) RegisterWorkflow(w interface{}) { + env.registry.RegisterWorkflow(w) +} + +func (env *testWorkflowEnvironmentImpl) RegisterWorkflowWithOptions(w interface{}, options RegisterWorkflowOptions) { + env.registry.RegisterWorkflowWithOptions(w, options) +} + +func (env *testWorkflowEnvironmentImpl) RegisterActivity(a interface{}) { + env.registry.RegisterActivityWithOptions(a, RegisterActivityOptions{DisableAlreadyRegisteredCheck: true}) +} + +func (env *testWorkflowEnvironmentImpl) RegisterActivityWithOptions(a interface{}, options RegisterActivityOptions) { + options.DisableAlreadyRegisteredCheck = true + env.registry.RegisterActivityWithOptions(a, options) +} + +func (env *testWorkflowEnvironmentImpl) RegisterNexusService(s *nexus.Service) { + env.registry.RegisterNexusService(s) +} + +func (env *testWorkflowEnvironmentImpl) RegisterCancelHandler(handler func()) { + env.workflowCancelHandler = handler +} + +func (env *testWorkflowEnvironmentImpl) RegisterSignalHandler( + handler func(name string, input *commonpb.Payloads, header *commonpb.Header) error, +) { + env.signalHandler = handler +} + +func (env *testWorkflowEnvironmentImpl) RegisterUpdateHandler( + handler func(name string, id string, input *commonpb.Payloads, header *commonpb.Header, resp UpdateCallbacks), +) { + env.updateHandler = handler +} + +func (env *testWorkflowEnvironmentImpl) RegisterQueryHandler( + handler func(string, *commonpb.Payloads, *commonpb.Header) (*commonpb.Payloads, error), +) { + env.queryHandler = handler +} + +func (env *testWorkflowEnvironmentImpl) RequestCancelChildWorkflow(_, workflowID string) { + if childHandle, ok := env.runningWorkflows[workflowID]; ok && !childHandle.handled { + // current workflow is a parent workflow, and we are canceling a child workflow + childEnv := childHandle.env + childEnv.cancelWorkflow(func(result *commonpb.Payloads, err error) {}) + return + } +} + +func (env *testWorkflowEnvironmentImpl) RequestCancelExternalWorkflow(namespace, workflowID, runID string, callback ResultHandler) { + if env.workflowInfo.WorkflowExecution.ID == workflowID { + // cancel current workflow + env.workflowCancelHandler() + // check if current workflow is a child workflow + if env.isChildWorkflow() && env.onChildWorkflowCanceledListener != nil { + env.postCallback(func() { + env.onChildWorkflowCanceledListener(env.workflowInfo) + }, false) + } + return + } else if childHandle, ok := env.runningWorkflows[workflowID]; ok && !childHandle.handled { + // current workflow is a parent workflow, and we are canceling a child workflow + if !childHandle.params.WaitForCancellation { + childHandle.env.Complete(nil, ErrCanceled) + } + childEnv := childHandle.env + env.postCallback(func() { + callback(nil, nil) + }, true) + childEnv.cancelWorkflow(callback) + return + } + + // target workflow is not child workflow, we need the mock. The mock needs to be called in a separate goroutinue + // so it can block and wait on the requested delay time (if configured). If we run it in main thread, and the mock + // configured to delay, it will block the main loop which stops the world. + env.runningCount++ + go func() { + args := []interface{}{namespace, workflowID, runID} + // below call will panic if mock is not properly setup. + mockRet := env.workflowMock.MethodCalled(mockMethodForRequestCancelExternalWorkflow, args...) + m := &mockWrapper{name: mockMethodForRequestCancelExternalWorkflow, fn: mockFnRequestCancelExternalWorkflow} + var err error + if mockFn := m.getMockFn(mockRet); mockFn != nil { + _, err = executeFunctionWithContext(context.TODO(), mockFn, args) + } else { + _, err = m.getMockValue(mockRet) + } + env.postCallback(func() { + callback(nil, err) + env.runningCount-- + }, true) + }() +} + +func (env *testWorkflowEnvironmentImpl) IsReplaying() bool { + // this test environment never replay + return false +} + +func (env *testWorkflowEnvironmentImpl) SignalExternalWorkflow( + namespace string, + workflowID string, + runID string, + signalName string, + input *commonpb.Payloads, + arg interface{}, + header *commonpb.Header, + childWorkflowOnly bool, + callback ResultHandler, +) { + // check if target workflow is a known workflow + if childHandle, ok := env.runningWorkflows[workflowID]; ok { + // target workflow is a child + childEnv := childHandle.env + if childEnv.isWorkflowCompleted { + // child already completed (NOTE: we have only one failed cause now) + err := newUnknownExternalWorkflowExecutionError() + callback(nil, err) + } else { + err := childEnv.signalHandler(signalName, input, header) + callback(nil, err) + } + childEnv.postCallback(func() {}, true) // resume child workflow since a signal is sent. + return + } + + // here we signal a child workflow but we cannot find it + if childWorkflowOnly { + err := newUnknownExternalWorkflowExecutionError() + callback(nil, err) + return + } + + // target workflow is not child workflow, we need the mock. The mock needs to be called in a separate goroutinue + // so it can block and wait on the requested delay time (if configured). If we run it in main thread, and the mock + // configured to delay, it will block the main loop which stops the world. + env.runningCount++ + go func() { + args := []interface{}{namespace, workflowID, runID, signalName, arg} + // below call will panic if mock is not properly setup. + mockRet := env.workflowMock.MethodCalled(mockMethodForSignalExternalWorkflow, args...) + m := &mockWrapper{name: mockMethodForSignalExternalWorkflow, fn: mockFnSignalExternalWorkflow} + var err error + if mockFn := m.getMockFn(mockRet); mockFn != nil { + _, err = executeFunctionWithContext(context.TODO(), mockFn, args) + } else { + _, err = m.getMockValue(mockRet) + } + env.postCallback(func() { + callback(nil, err) + env.runningCount-- + }, true) + }() +} + +func (env *testWorkflowEnvironmentImpl) ExecuteChildWorkflow(params ExecuteWorkflowParams, callback ResultHandler, startedHandler func(r WorkflowExecution, e error)) { + env.executeChildWorkflowWithDelay(0, params, callback, startedHandler) +} + +func (env *testWorkflowEnvironmentImpl) executeChildWorkflowWithDelay(delayStart time.Duration, params ExecuteWorkflowParams, callback ResultHandler, startedHandler func(r WorkflowExecution, e error)) { + childEnv, err := env.newTestWorkflowEnvironmentForChild(¶ms, callback, startedHandler) + if err != nil { + env.logger.Info("ExecuteChildWorkflow failed", tagError, err) + callback(nil, err) + startedHandler(WorkflowExecution{}, err) + return + } + + env.logger.Info("ExecuteChildWorkflow", tagWorkflowType, params.WorkflowType.Name) + env.runningCount++ + + // run child workflow in separate goroutinue + go childEnv.executeWorkflowInternal(delayStart, params.WorkflowType.Name, params.Input) +} + +func (env *testWorkflowEnvironmentImpl) newTestNexusTaskHandler() *nexusTaskHandler { + if len(env.registry.nexusServices) == 0 { + panic(fmt.Errorf("no nexus services registered")) + } + + reg := nexus.NewServiceRegistry() + for _, service := range env.registry.nexusServices { + if err := reg.Register(service); err != nil { + panic(fmt.Errorf("failed to register nexus service '%v': %w", service, err)) + } + } + handler, err := reg.NewHandler() + if err != nil { + panic(fmt.Errorf("failed to create nexus handler: %w", err)) + } + + return newNexusTaskHandler( + handler, + env.identity, + env.workflowInfo.Namespace, + env.workflowInfo.TaskQueueName, + &testSuiteClientForNexusOperations{env: env}, + env.dataConverter, + env.logger, + env.metricsHandler, + ) +} + +func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation(params executeNexusOperationParams, callback func(*commonpb.Payload, error), startedHandler func(opID string, e error)) int64 { + seq := env.nextID() + taskHandler := env.newTestNexusTaskHandler() + handle := &testNexusOperationHandle{ + env: env, + seq: seq, + params: params, + onCompleted: callback, + onStarted: startedHandler, + } + env.runningNexusOperations[seq] = handle + + task := handle.newStartTask() + env.runningCount++ + go func() { + response, failure, err := taskHandler.Execute(task) + if err != nil { + // No retries for operations, fail the operation immediately. + failure = taskHandler.fillInFailure(task.TaskToken, nexusHandlerError(nexus.HandlerErrorTypeInternal, err.Error())) + } + if failure != nil { + err := env.failureConverter.FailureToError(nexusOperationFailure(params, "", &failurepb.Failure{ + Message: failure.GetError().GetFailure().GetMessage(), + FailureInfo: &failurepb.Failure_ApplicationFailureInfo{ + ApplicationFailureInfo: &failurepb.ApplicationFailureInfo{ + NonRetryable: true, + }, + }, + })) + env.postCallback(func() { + handle.startedCallback("", err) + handle.completedCallback(nil, err) + }, true) + return + } else { + switch v := response.GetResponse().GetStartOperation().GetVariant().(type) { + case *nexuspb.StartOperationResponse_SyncSuccess: + env.postCallback(func() { + handle.startedCallback("", nil) + handle.completedCallback(v.SyncSuccess.GetPayload(), nil) + }, true) + case *nexuspb.StartOperationResponse_AsyncSuccess: + env.postCallback(func() { + handle.startedCallback(v.AsyncSuccess.GetOperationId(), nil) + if handle.cancelRequested { + handle.cancel() + } + }, true) + case *nexuspb.StartOperationResponse_OperationError: + err := env.failureConverter.FailureToError( + nexusOperationFailure(params, "", unsuccessfulOperationErrorToTemporalFailure(v.OperationError)), + ) + env.postCallback(func() { + handle.startedCallback("", err) + handle.completedCallback(nil, err) + }, true) + default: + panic(fmt.Errorf("unknown response variant: %v", v)) + } + } + }() + return seq +} + +func (env *testWorkflowEnvironmentImpl) RequestCancelNexusOperation(seq int64) { + handle, ok := env.runningNexusOperations[seq] + if !ok { + panic(fmt.Errorf("no running operation found for sequence: %d", seq)) + } + + // Avoid duplicate cancelation. + if handle.cancelRequested { + return + } + + // Mark this cancelation request in case the operation hasn't started yet. + // Cancel will be called after start. + handle.cancelRequested = true + + // Only cancel after started, we need an operation ID. + if handle.started { + handle.cancel() + } +} + +func (env *testWorkflowEnvironmentImpl) resolveNexusOperation(seq int64, result *commonpb.Payload, err error) { + env.postCallback(func() { + handle, ok := env.runningNexusOperations[seq] + if !ok { + panic(fmt.Errorf("no running operation found for sequence: %d", seq)) + } + if err != nil { + failure := env.failureConverter.ErrorToFailure(err) + err = env.failureConverter.FailureToError(nexusOperationFailure(handle.params, handle.operationID, failure.GetCause())) + handle.completedCallback(nil, err) + } else { + handle.completedCallback(result, nil) + } + }, true) +} + +func (env *testWorkflowEnvironmentImpl) SideEffect(f func() (*commonpb.Payloads, error), callback ResultHandler) { + callback(f()) +} + +func (env *testWorkflowEnvironmentImpl) GetVersion(changeID string, minSupported, maxSupported Version) (retVersion Version) { + if mockVersion, ok := env.getMockedVersion(changeID, changeID, minSupported, maxSupported); ok { + // GetVersion for changeID is mocked + _ = env.UpsertSearchAttributes(createSearchAttributesForChangeVersion(changeID, mockVersion, env.changeVersions)) + env.changeVersions[changeID] = mockVersion + return mockVersion + } + if mockVersion, ok := env.getMockedVersion(mock.Anything, changeID, minSupported, maxSupported); ok { + // GetVersion is mocked with any changeID. + _ = env.UpsertSearchAttributes(createSearchAttributesForChangeVersion(changeID, mockVersion, env.changeVersions)) + env.changeVersions[changeID] = mockVersion + return mockVersion + } + + // no mock setup, so call regular path + if version, ok := env.changeVersions[changeID]; ok { + validateVersion(changeID, version, minSupported, maxSupported) + return version + } + _ = env.UpsertSearchAttributes(createSearchAttributesForChangeVersion(changeID, maxSupported, env.changeVersions)) + env.changeVersions[changeID] = maxSupported + return maxSupported +} + +func (env *testWorkflowEnvironmentImpl) getMockedVersion(mockedChangeID, changeID string, minSupported, maxSupported Version) (Version, bool) { + mockMethod := getMockMethodForGetVersion(mockedChangeID) + if _, ok := env.expectedWorkflowMockCalls[mockMethod]; !ok { + // mock not found + return DefaultVersion, false + } + + args := []interface{}{changeID, minSupported, maxSupported} + // below call will panic if mock is not properly setup. + mockRet := env.workflowMock.MethodCalled(mockMethod, args...) + m := &mockWrapper{name: mockMethodForGetVersion, fn: mockFnGetVersion} + if mockFn := m.getMockFn(mockRet); mockFn != nil { + var reflectArgs []reflect.Value + // Add context if first param + if fnType := reflect.TypeOf(mockFn); fnType.NumIn() > 0 && isActivityContext(fnType.In(0)) { + reflectArgs = append(reflectArgs, reflect.ValueOf(context.TODO())) + } + for _, arg := range args { + reflectArgs = append(reflectArgs, reflect.ValueOf(arg)) + } + reflectValues := reflect.ValueOf(mockFn).Call(reflectArgs) + if len(reflectValues) != 1 || !reflect.TypeOf(reflectValues[0].Interface()).AssignableTo(reflect.TypeOf(DefaultVersion)) { + panic(fmt.Sprintf("mock of GetVersion has incorrect return type, expected workflow.Version, but actual is %T (%v)", + reflectValues[0].Interface(), reflectValues[0].Interface())) + } + return reflectValues[0].Interface().(Version), true + } + + if len(mockRet) != 1 || !reflect.TypeOf(mockRet[0]).AssignableTo(reflect.TypeOf(DefaultVersion)) { + panic(fmt.Sprintf("mock of GetVersion has incorrect return type, expected workflow.Version, but actual is %T (%v)", + mockRet[0], mockRet[0])) + } + return mockRet[0].(Version), true +} + +func getMockMethodForGetVersion(changeID string) string { + return fmt.Sprintf("%v_%v", mockMethodForGetVersion, changeID) +} + +func (env *testWorkflowEnvironmentImpl) UpsertSearchAttributes(attributes map[string]interface{}) error { + attr, err := validateAndSerializeSearchAttributes(attributes) + + env.workflowInfo.SearchAttributes = mergeSearchAttributes(env.workflowInfo.SearchAttributes, attr) + + mockMethod := mockMethodForUpsertSearchAttributes + if _, ok := env.expectedWorkflowMockCalls[mockMethod]; !ok { + // mock not found + return err + } + + args := []interface{}{attributes} + env.workflowMock.MethodCalled(mockMethod, args...) + + return err +} + +func validateAndSerializeTypedSearchAttributes(searchAttributes map[SearchAttributeKey]interface{}) (*commonpb.SearchAttributes, error) { + if len(searchAttributes) == 0 { + return nil, errSearchAttributesNotSet + } + + rawSearchAttributes, err := serializeTypedSearchAttributes(searchAttributes) + if err != nil { + return nil, err + } + + return rawSearchAttributes, nil +} + +func (env *testWorkflowEnvironmentImpl) UpsertTypedSearchAttributes(attributes SearchAttributes) error { + // Don't immediately return the error from validateAndSerializeTypedSearchAttributes, as we may need to call the mock + rawSearchAttributes, err := validateAndSerializeTypedSearchAttributes(attributes.untypedValue) + + env.workflowInfo.SearchAttributes = mergeSearchAttributes(env.workflowInfo.SearchAttributes, rawSearchAttributes) + + mockMethod := mockMethodForUpsertTypedSearchAttributes + if _, ok := env.expectedWorkflowMockCalls[mockMethod]; !ok { + // mock not found + return err + } + + args := []interface{}{attributes} + env.workflowMock.MethodCalled(mockMethod, args...) + + return err +} + +func (env *testWorkflowEnvironmentImpl) UpsertMemo(memoMap map[string]interface{}) error { + memo, err := validateAndSerializeMemo(memoMap, env.dataConverter) + + env.workflowInfo.Memo = mergeMemo(env.workflowInfo.Memo, memo) + + mockMethod := mockMethodForUpsertMemo + if _, ok := env.expectedWorkflowMockCalls[mockMethod]; !ok { + // mock not found + return err + } + + args := []interface{}{memoMap} + env.workflowMock.MethodCalled(mockMethod, args...) + + return err +} + +func (env *testWorkflowEnvironmentImpl) MutableSideEffect(_ string, f func() interface{}, _ func(a, b interface{}) bool) converter.EncodedValue { + return newEncodedValue(env.encodeValue(f()), env.GetDataConverter()) +} + +func (env *testWorkflowEnvironmentImpl) AddSession(sessionInfo *SessionInfo) { + env.openSessions[sessionInfo.SessionID] = sessionInfo +} + +func (env *testWorkflowEnvironmentImpl) RemoveSession(sessionID string) { + delete(env.openSessions, sessionID) +} + +func (env *testWorkflowEnvironmentImpl) encodeValue(value interface{}) *commonpb.Payloads { + blob, err := env.GetDataConverter().ToPayloads(value) + if err != nil { + panic(err) + } + return blob +} + +func (env *testWorkflowEnvironmentImpl) nextID() int64 { + activityID := env.counterID + env.counterID++ + return activityID +} + +func (env *testWorkflowEnvironmentImpl) getActivityInfo(activityID ActivityID, activityType string) *ActivityInfo { + return &ActivityInfo{ + ActivityID: activityID.id, + ActivityType: ActivityType{Name: activityType}, + TaskToken: []byte(activityID.id), + WorkflowExecution: env.workflowInfo.WorkflowExecution, + Attempt: 1, + } +} + +func (env *testWorkflowEnvironmentImpl) cancelWorkflow(callback ResultHandler) { + env.cancelWorkflowByID(env.workflowInfo.WorkflowExecution.ID, env.workflowInfo.WorkflowExecution.RunID, callback) +} + +func (env *testWorkflowEnvironmentImpl) cancelWorkflowByID(workflowID string, runID string, callback ResultHandler) { + env.postCallback(func() { + // RequestCancelWorkflow needs to be run in main thread + env.RequestCancelExternalWorkflow( + env.workflowInfo.Namespace, + workflowID, + runID, + callback, + ) + }, true) +} + +func (env *testWorkflowEnvironmentImpl) signalWorkflow(name string, input interface{}, startWorkflowTask bool) { + data, err := encodeArg(env.GetDataConverter(), input) + if err != nil { + panic(err) + } + env.postCallback(func() { + // Do not send any headers on test invocations + _ = env.signalHandler(name, data, nil) + }, startWorkflowTask) +} + +func (env *testWorkflowEnvironmentImpl) signalWorkflowByID(workflowID, signalName string, input interface{}) error { + data, err := encodeArg(env.GetDataConverter(), input) + if err != nil { + panic(err) + } + + if workflowHandle, ok := env.runningWorkflows[workflowID]; ok { + if workflowHandle.handled { + return serviceerror.NewNotFound(fmt.Sprintf("Workflow %v already completed", workflowID)) + } + workflowHandle.env.postCallback(func() { + // Do not send any headers on test invocations + _ = workflowHandle.env.signalHandler(signalName, data, nil) + }, true) + return nil + } + + return serviceerror.NewNotFound(fmt.Sprintf("Workflow %v not exists", workflowID)) +} + +func (env *testWorkflowEnvironmentImpl) queryWorkflow(queryType string, args ...interface{}) (converter.EncodedValue, error) { + data, err := encodeArgs(env.GetDataConverter(), args) + if err != nil { + return nil, err + } + // Do not send any headers on test invocations + blob, err := env.queryHandler(queryType, data, nil) + if err != nil { + return nil, err + } + return newEncodedValue(blob, env.GetDataConverter()), nil +} + +func (env *testWorkflowEnvironmentImpl) updateWorkflow(name string, id string, uc UpdateCallbacks, args ...interface{}) { + data, err := encodeArgs(env.GetDataConverter(), args) + if err != nil { + panic(err) + } + env.postCallback(func() { + // Do not send any headers on test invocations + env.updateHandler(name, id, data, nil, uc) + }, true) +} + +func (env *testWorkflowEnvironmentImpl) updateWorkflowByID(workflowID, name, id string, uc UpdateCallbacks, args ...interface{}) error { + if workflowHandle, ok := env.runningWorkflows[workflowID]; ok { + if workflowHandle.handled { + return serviceerror.NewNotFound(fmt.Sprintf("Workflow %v already completed", workflowID)) + } + data, err := encodeArgs(env.GetDataConverter(), args) + if err != nil { + panic(err) + } + workflowHandle.env.postCallback(func() { + workflowHandle.env.updateHandler(name, id, data, nil, uc) + }, true) + return nil + } + + return serviceerror.NewNotFound(fmt.Sprintf("Workflow %v not exists", workflowID)) +} + +func (env *testWorkflowEnvironmentImpl) queryWorkflowByID(workflowID, queryType string, args ...interface{}) (converter.EncodedValue, error) { + if workflowHandle, ok := env.runningWorkflows[workflowID]; ok { + data, err := encodeArgs(workflowHandle.env.GetDataConverter(), args) + if err != nil { + return nil, err + } + // Do not send any headers on test invocations + blob, err := workflowHandle.env.queryHandler(queryType, data, nil) + if err != nil { + return nil, err + } + return newEncodedValue(blob, workflowHandle.env.GetDataConverter()), nil + } + return nil, serviceerror.NewNotFound(fmt.Sprintf("Workflow %v not exists", workflowID)) +} + +func (env *testWorkflowEnvironmentImpl) getWorkflowMockRunFn(callWrapper *MockCallWrapper) func(args mock.Arguments) { + env.locker.Lock() + defer env.locker.Unlock() + + env.expectedWorkflowMockCalls[callWrapper.call.Method] = struct{}{} + return func(args mock.Arguments) { + env.runBeforeMockCallReturns(callWrapper, args) + } +} + +func (env *testWorkflowEnvironmentImpl) getActivityMockRunFn(callWrapper *MockCallWrapper) func(args mock.Arguments) { + env.locker.Lock() + defer env.locker.Unlock() + + env.expectedActivityMockCalls[callWrapper.call.Method] = struct{}{} + return func(args mock.Arguments) { + env.runBeforeMockCallReturns(callWrapper, args) + } +} + +func (env *testWorkflowEnvironmentImpl) setLastCompletionResult(result interface{}) { + data, err := encodeArg(env.GetDataConverter(), result) + if err != nil { + panic(err) + } + env.workflowInfo.lastCompletionResult = data +} + +func (env *testWorkflowEnvironmentImpl) setLastError(err error) { + env.workflowInfo.lastFailure = env.failureConverter.ErrorToFailure(err) +} + +func (env *testWorkflowEnvironmentImpl) setHeartbeatDetails(details interface{}) { + data, err := encodeArg(env.GetDataConverter(), details) + if err != nil { + panic(err) + } + env.heartbeatDetails = data +} + +func (env *testWorkflowEnvironmentImpl) GetRegistry() *registry { + return env.registry +} + +func (env *testWorkflowEnvironmentImpl) setStartWorkflowOptions(options StartWorkflowOptions) { + wf := env.workflowInfo + if options.WorkflowExecutionTimeout > 0 { + wf.WorkflowExecutionTimeout = options.WorkflowExecutionTimeout + } + if options.WorkflowRunTimeout > 0 { + wf.WorkflowRunTimeout = options.WorkflowRunTimeout + } + if options.WorkflowTaskTimeout > 0 { + wf.WorkflowTaskTimeout = options.WorkflowTaskTimeout + } + if len(options.ID) > 0 { + // Reassign the ID in running Workflows so SignalWorkflowByID can find the workflow + originalID := wf.WorkflowExecution.ID + env.runningWorkflows[options.ID] = env.runningWorkflows[wf.WorkflowExecution.ID] + delete(env.runningWorkflows, originalID) + wf.WorkflowExecution.ID = options.ID + } + if len(options.TaskQueue) > 0 { + wf.TaskQueueName = options.TaskQueue + } +} + +func newTestSessionEnvironment(testWorkflowEnvironment *testWorkflowEnvironmentImpl, + params *workerExecutionParameters, concurrentSessionExecutionSize int) *testSessionEnvironmentImpl { + resourceID := params.SessionResourceID + if resourceID == "" { + resourceID = "testResourceID" + } + if concurrentSessionExecutionSize == 0 { + concurrentSessionExecutionSize = defaultMaxConcurrentSessionExecutionSize + } + + return &testSessionEnvironmentImpl{ + sessionEnvironmentImpl: newSessionEnvironment(resourceID, concurrentSessionExecutionSize).(*sessionEnvironmentImpl), + testWorkflowEnvironment: testWorkflowEnvironment, + } +} + +func (t *testSessionEnvironmentImpl) SignalCreationResponse(_ context.Context, sessionID string) error { + t.testWorkflowEnvironment.signalWorkflow(sessionID, t.sessionEnvironmentImpl.getCreationResponse(), true) + return nil +} + +// function signature for mock SignalExternalWorkflow +func mockFnSignalExternalWorkflow(string, string, string, string, interface{}) error { + return nil +} + +// function signature for mock RequestCancelExternalWorkflow +func mockFnRequestCancelExternalWorkflow(string, string, string) error { + return nil +} + +// function signature for mock GetVersion +func mockFnGetVersion(string, Version, Version) Version { + return DefaultVersion +} + +// make sure interface is implemented +var _ WorkflowEnvironment = (*testWorkflowEnvironmentImpl)(nil) + +func (h *testNexusOperationHandle) newStartTask() *workflowservice.PollNexusTaskQueueResponse { + return &workflowservice.PollNexusTaskQueueResponse{ + TaskToken: []byte{}, + Request: &nexuspb.Request{ + ScheduledTime: timestamppb.Now(), + Header: h.params.nexusHeader, + Variant: &nexuspb.Request_StartOperation{ + StartOperation: &nexuspb.StartOperationRequest{ + Service: h.params.client.Service(), + Operation: h.params.operation, + RequestId: uuid.NewString(), + // This is effectively ignored. + Callback: "http://test-env/operations", + CallbackHeader: map[string]string{ + // The test client uses this to call resolveNexusOperation. + "operation-sequence": strconv.FormatInt(h.seq, 10), + }, + Payload: h.params.input, + }, + }, + }, + } +} + +func (h *testNexusOperationHandle) newCancelTask() *workflowservice.PollNexusTaskQueueResponse { + return &workflowservice.PollNexusTaskQueueResponse{ + TaskToken: []byte{}, + Request: &nexuspb.Request{ + ScheduledTime: timestamppb.Now(), + Header: h.params.nexusHeader, + Variant: &nexuspb.Request_CancelOperation{ + CancelOperation: &nexuspb.CancelOperationRequest{ + Service: h.params.client.Service(), + Operation: h.params.operation, + OperationId: h.operationID, + }, + }, + }, + } +} + +// completedCallback is a callback registered to handle operation completion. +// Must be called in a postCallback block. +func (h *testNexusOperationHandle) completedCallback(result *commonpb.Payload, err error) { + if h.done { + // Ignore duplicate completions. + return + } + h.done = true + delete(h.env.runningNexusOperations, h.seq) + h.onCompleted(result, err) +} + +// startedCallback is a callback registered to handle operation start. +// Must be called in a postCallback block. +func (h *testNexusOperationHandle) startedCallback(opID string, e error) { + h.operationID = opID + h.started = true + h.onStarted(opID, e) + h.env.runningCount-- +} + +func (h *testNexusOperationHandle) cancel() { + if h.done { + return + } + if h.started && h.operationID == "" { + panic(fmt.Errorf("incomplete operation has no operation ID: (%s, %s, %s)", + h.params.client.Endpoint(), h.params.client.Service(), h.params.operation)) + } + h.env.runningCount++ + task := h.newCancelTask() + taskHandler := h.env.newTestNexusTaskHandler() + + go func() { + _, failure, err := taskHandler.Execute(task) + h.env.postCallback(func() { + if err != nil { + // No retries in the test env, fail the operation immediately. + h.completedCallback(nil, fmt.Errorf("operation cancelation handler failed: %w", err)) + } else if failure != nil { + // No retries in the test env, fail the operation immediately. + h.completedCallback(nil, fmt.Errorf("operation cancelation handler failed: %v", failure.GetError().GetFailure().GetMessage())) + } + h.env.runningCount-- + }, false) + }() +} diff --git a/vendor/go.temporal.io/sdk/internal/log/default_logger.go b/vendor/go.temporal.io/sdk/internal/log/default_logger.go new file mode 100644 index 00000000000..351b10945cb --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/log/default_logger.go @@ -0,0 +1,89 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package log + +import ( + "fmt" + golog "log" + "os" + "strings" + + "go.temporal.io/sdk/log" +) + +// DefaultLogger is Logger implementation on top of standard log.Logger. It is used if logger is not specified. +type DefaultLogger struct { + logger *golog.Logger + globalKeyvals string +} + +// NewDefaultLogger creates new instance of DefaultLogger. +func NewDefaultLogger() *DefaultLogger { + return &DefaultLogger{logger: golog.New(os.Stdout, "", golog.LstdFlags)} +} + +func (l *DefaultLogger) println(level, msg string, keyvals []interface{}) { + // To avoid extra space when globalKeyvals is not specified. + if l.globalKeyvals == "" { + l.logger.Println(append([]interface{}{level, msg}, keyvals...)...) + } else { + l.logger.Println(append([]interface{}{level, msg, l.globalKeyvals}, keyvals...)...) + } +} + +// Debug writes message to the log. +func (l *DefaultLogger) Debug(msg string, keyvals ...interface{}) { + l.println("DEBUG", msg, keyvals) +} + +// Info writes message to the log. +func (l *DefaultLogger) Info(msg string, keyvals ...interface{}) { + l.println("INFO ", msg, keyvals) +} + +// Warn writes message to the log. +func (l *DefaultLogger) Warn(msg string, keyvals ...interface{}) { + l.println("WARN ", msg, keyvals) +} + +// Error writes message to the log. +func (l *DefaultLogger) Error(msg string, keyvals ...interface{}) { + l.println("ERROR", msg, keyvals) +} + +// With returns new logger the prepend every log entry with keyvals. +func (l *DefaultLogger) With(keyvals ...interface{}) log.Logger { + logger := &DefaultLogger{ + logger: l.logger, + } + + if l.globalKeyvals != "" { + logger.globalKeyvals = l.globalKeyvals + " " + } + + logger.globalKeyvals += strings.TrimSuffix(fmt.Sprintln(keyvals...), "\n") + + return logger +} diff --git a/vendor/go.temporal.io/sdk/internal/log/memory_logger.go b/vendor/go.temporal.io/sdk/internal/log/memory_logger.go new file mode 100644 index 00000000000..c0a026fa466 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/log/memory_logger.go @@ -0,0 +1,106 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package log + +import ( + "fmt" + "strings" + + "go.temporal.io/sdk/log" +) + +// MemoryLoggerWithoutWith is a Logger implementation that stores logs in memory (useful for testing). Use Lines() to get log lines. +type MemoryLoggerWithoutWith struct { + lines *[]string + globalKeyvals string +} + +// NewMemoryLoggerWithoutWith creates new instance of MemoryLoggerWithoutWith. +func NewMemoryLoggerWithoutWith() *MemoryLoggerWithoutWith { + var lines []string + return &MemoryLoggerWithoutWith{ + lines: &lines, + } +} + +func (l *MemoryLoggerWithoutWith) println(level, msg string, keyvals []interface{}) { + // To avoid extra space when globalKeyvals is not specified. + if l.globalKeyvals == "" { + *l.lines = append(*l.lines, fmt.Sprintln(append([]interface{}{level, msg}, keyvals...)...)) + } else { + *l.lines = append(*l.lines, fmt.Sprintln(append([]interface{}{level, msg, l.globalKeyvals}, keyvals...)...)) + } +} + +// Debug appends message to the log. +func (l *MemoryLoggerWithoutWith) Debug(msg string, keyvals ...interface{}) { + l.println("DEBUG", msg, keyvals) +} + +// Info appends message to the log. +func (l *MemoryLoggerWithoutWith) Info(msg string, keyvals ...interface{}) { + l.println("INFO ", msg, keyvals) +} + +// Warn appends message to the log. +func (l *MemoryLoggerWithoutWith) Warn(msg string, keyvals ...interface{}) { + l.println("WARN ", msg, keyvals) +} + +// Error appends message to the log. +func (l *MemoryLoggerWithoutWith) Error(msg string, keyvals ...interface{}) { + l.println("ERROR", msg, keyvals) +} + +// Lines returns written log lines. +func (l *MemoryLoggerWithoutWith) Lines() []string { + return *l.lines +} + +type MemoryLogger struct { + *MemoryLoggerWithoutWith +} + +// NewMemoryLogger creates new instance of MemoryLogger. +func NewMemoryLogger() *MemoryLogger { + return &MemoryLogger{ + NewMemoryLoggerWithoutWith(), + } +} + +// With returns new logger that prepend every log entry with keyvals. +func (l *MemoryLogger) With(keyvals ...interface{}) log.Logger { + logger := &MemoryLoggerWithoutWith{ + lines: l.lines, + } + + if l.globalKeyvals != "" { + logger.globalKeyvals = l.globalKeyvals + " " + } + + logger.globalKeyvals += strings.TrimSuffix(fmt.Sprintln(keyvals...), "\n") + + return logger +} diff --git a/vendor/go.temporal.io/sdk/internal/log/noop_logger.go b/vendor/go.temporal.io/sdk/internal/log/noop_logger.go new file mode 100644 index 00000000000..9be04910b36 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/log/noop_logger.go @@ -0,0 +1,55 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package log + +import ( + "go.temporal.io/sdk/log" +) + +// NoopLogger is Logger implementation that doesn't produce any logs. +type NoopLogger struct { +} + +// NewNopLogger creates new instance of NoopLogger. +func NewNopLogger() *NoopLogger { + return &NoopLogger{} +} + +// Debug does nothing. +func (l *NoopLogger) Debug(string, ...interface{}) {} + +// Info does nothing. +func (l *NoopLogger) Info(string, ...interface{}) {} + +// Warn does nothing. +func (l *NoopLogger) Warn(string, ...interface{}) {} + +// Error does nothing. +func (l *NoopLogger) Error(string, ...interface{}) {} + +// With returns new NoopLogger. +func (l *NoopLogger) With(...interface{}) log.Logger { + return NewNopLogger() +} diff --git a/vendor/go.temporal.io/sdk/internal/log/replay_logger.go b/vendor/go.temporal.io/sdk/internal/log/replay_logger.go new file mode 100644 index 00000000000..459d94249a3 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/log/replay_logger.go @@ -0,0 +1,93 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package log + +import ( + "go.temporal.io/sdk/log" +) + +var _ log.Logger = (*ReplayLogger)(nil) +var _ log.WithLogger = (*ReplayLogger)(nil) +var _ log.WithSkipCallers = (*ReplayLogger)(nil) + +// ReplayLogger is Logger implementation that is aware of replay. +type ReplayLogger struct { + logger log.Logger + isReplay *bool // pointer to bool that indicate if it is in replay mode + enableLoggingInReplay *bool // pointer to bool that indicate if logging is enabled in replay mode +} + +// NewReplayLogger crates new instance of ReplayLogger. +func NewReplayLogger(logger log.Logger, isReplay *bool, enableLoggingInReplay *bool) log.Logger { + return &ReplayLogger{ + logger: logger, + isReplay: isReplay, + enableLoggingInReplay: enableLoggingInReplay, + } +} + +func (l *ReplayLogger) check() bool { + return !*l.isReplay || *l.enableLoggingInReplay +} + +// Debug writes message to the log if it is not a replay. +func (l *ReplayLogger) Debug(msg string, keyvals ...interface{}) { + if l.check() { + l.logger.Debug(msg, keyvals...) + } +} + +// Info writes message to the log if it is not a replay. +func (l *ReplayLogger) Info(msg string, keyvals ...interface{}) { + if l.check() { + l.logger.Info(msg, keyvals...) + } +} + +// Warn writes message to the log if it is not a replay. +func (l *ReplayLogger) Warn(msg string, keyvals ...interface{}) { + if l.check() { + l.logger.Warn(msg, keyvals...) + } +} + +// Error writes message to the log if it is not a replay. +func (l *ReplayLogger) Error(msg string, keyvals ...interface{}) { + if l.check() { + l.logger.Error(msg, keyvals...) + } +} + +// With returns new logger that prepend every log entry with keyvals. +func (l *ReplayLogger) With(keyvals ...interface{}) log.Logger { + return NewReplayLogger(log.With(l.logger, keyvals...), l.isReplay, l.enableLoggingInReplay) +} + +func (l *ReplayLogger) WithCallerSkip(depth int) log.Logger { + if sl, ok := l.logger.(log.WithSkipCallers); ok { + return NewReplayLogger(sl.WithCallerSkip(depth), l.isReplay, l.enableLoggingInReplay) + } + return l +} diff --git a/vendor/go.temporal.io/sdk/internal/log/test_reporter.go b/vendor/go.temporal.io/sdk/internal/log/test_reporter.go new file mode 100644 index 00000000000..1595372aba5 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/log/test_reporter.go @@ -0,0 +1,53 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package log + +import ( + "fmt" + "os" + + "go.temporal.io/sdk/log" +) + +// TestReporter is a log adapter for gomock. +type TestReporter struct { + logger log.Logger +} + +// NewTestReporter creates new instance of TestReporter. +func NewTestReporter(logger log.Logger) *TestReporter { + return &TestReporter{logger: logger} +} + +// Errorf writes error to the log. +func (t *TestReporter) Errorf(format string, args ...interface{}) { + t.logger.Error(fmt.Sprintf(format, args...)) +} + +// Fatalf writes error to the log and exits. +func (t *TestReporter) Fatalf(format string, args ...interface{}) { + t.logger.Error(fmt.Sprintf(format, args...)) + os.Exit(1) +} diff --git a/vendor/go.temporal.io/sdk/internal/nexus_operations.go b/vendor/go.temporal.io/sdk/internal/nexus_operations.go new file mode 100644 index 00000000000..80bd4e98a1b --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/nexus_operations.go @@ -0,0 +1,425 @@ +// The MIT License +// +// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "fmt" + "strconv" + + "github.com/nexus-rpc/sdk-go/nexus" + commonpb "go.temporal.io/api/common/v1" + "go.temporal.io/api/enums/v1" + failurepb "go.temporal.io/api/failure/v1" + nexuspb "go.temporal.io/api/nexus/v1" + "go.temporal.io/api/operatorservice/v1" + "go.temporal.io/api/workflowservice/v1" + + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/log" +) + +// NexusOperationContext is an internal only struct that holds fields used by the temporalnexus functions. +type NexusOperationContext struct { + Client Client + Namespace string + TaskQueue string + MetricsHandler metrics.Handler + Log log.Logger +} + +type nexusOperationContextKeyType struct{} + +// nexusOperationContextKey is a key for associating a [NexusOperationContext] with a [context.Context]. +var nexusOperationContextKey = nexusOperationContextKeyType{} + +type isWorkflowRunOpContextKeyType struct{} + +// IsWorkflowRunOpContextKey is a key to mark that the current context is used within a workflow run operation. +// The fake test env client verifies this key is set on the context to decide whether it should execute a method or +// panic as we don't want to expose a partial client to sync operations. +var IsWorkflowRunOpContextKey = isWorkflowRunOpContextKeyType{} + +// NexusOperationContextFromGoContext gets the [NexusOperationContext] associated with the given [context.Context]. +func NexusOperationContextFromGoContext(ctx context.Context) (nctx *NexusOperationContext, ok bool) { + nctx, ok = ctx.Value(nexusOperationContextKey).(*NexusOperationContext) + return +} + +// nexusOperationFailure is a utility in use by the test environment. +func nexusOperationFailure(params executeNexusOperationParams, operationID string, cause *failurepb.Failure) *failurepb.Failure { + return &failurepb.Failure{ + Message: "nexus operation completed unsuccessfully", + FailureInfo: &failurepb.Failure_NexusOperationExecutionFailureInfo{ + NexusOperationExecutionFailureInfo: &failurepb.NexusOperationFailureInfo{ + Endpoint: params.client.Endpoint(), + Service: params.client.Service(), + Operation: params.operation, + OperationId: operationID, + }, + }, + Cause: cause, + } +} + +// unsuccessfulOperationErrorToTemporalFailure is a utility in use by the test environment. +// copied from the server codebase with a slight adaptation: https://github.com/temporalio/temporal/blob/7635cd7dbdc7dd3219f387e8fc66fa117f585ff6/common/nexus/failure.go#L69-L108 +func unsuccessfulOperationErrorToTemporalFailure(err *nexuspb.UnsuccessfulOperationError) *failurepb.Failure { + failure := &failurepb.Failure{ + Message: err.Failure.Message, + } + if err.OperationState == string(nexus.OperationStateCanceled) { + failure.FailureInfo = &failurepb.Failure_CanceledFailureInfo{ + CanceledFailureInfo: &failurepb.CanceledFailureInfo{ + Details: nexusFailureMetadataToPayloads(err.Failure), + }, + } + } else { + failure.FailureInfo = &failurepb.Failure_ApplicationFailureInfo{ + ApplicationFailureInfo: &failurepb.ApplicationFailureInfo{ + // Make up a type here, it's not part of the Nexus Failure spec. + Type: "NexusOperationFailure", + Details: nexusFailureMetadataToPayloads(err.Failure), + NonRetryable: true, + }, + } + } + return failure +} + +// nexusFailureMetadataToPayloads is a utility in use by the test environment. +// copied from the server codebase with a slight adaptation: https://github.com/temporalio/temporal/blob/7635cd7dbdc7dd3219f387e8fc66fa117f585ff6/common/nexus/failure.go#L69-L108 +func nexusFailureMetadataToPayloads(failure *nexuspb.Failure) *commonpb.Payloads { + if len(failure.Metadata) == 0 && len(failure.Details) == 0 { + return nil + } + metadata := make(map[string][]byte, len(failure.Metadata)) + for k, v := range failure.Metadata { + metadata[k] = []byte(v) + } + return &commonpb.Payloads{ + Payloads: []*commonpb.Payload{ + { + Metadata: metadata, + Data: failure.Details, + }, + }, + } +} + +// testSuiteClientForNexusOperations is a partial [Client] implementation for the test workflow environment used to +// support running the workflow run operation - and only this operation, all methods will panic when this client is +// passed to sync operations. +type testSuiteClientForNexusOperations struct { + env *testWorkflowEnvironmentImpl +} + +// CancelWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) CancelWorkflow(ctx context.Context, workflowID string, runID string) error { + if set, ok := ctx.Value(IsWorkflowRunOpContextKey).(bool); !ok || !set { + panic("not implemented in the test environment") + } + doneCh := make(chan error) + t.env.cancelWorkflowByID(workflowID, runID, func(result *commonpb.Payloads, err error) { + doneCh <- err + }) + return <-doneCh +} + +// CheckHealth implements Client. +func (t *testSuiteClientForNexusOperations) CheckHealth(ctx context.Context, request *CheckHealthRequest) (*CheckHealthResponse, error) { + return &CheckHealthResponse{}, nil +} + +// Close implements Client. +func (t *testSuiteClientForNexusOperations) Close() { + // No op. +} + +// CompleteActivity implements Client. +func (t *testSuiteClientForNexusOperations) CompleteActivity(ctx context.Context, taskToken []byte, result interface{}, err error) error { + panic("not implemented in the test environment") +} + +// CompleteActivityByID implements Client. +func (t *testSuiteClientForNexusOperations) CompleteActivityByID(ctx context.Context, namespace string, workflowID string, runID string, activityID string, result interface{}, err error) error { + panic("not implemented in the test environment") +} + +// CountWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) CountWorkflow(ctx context.Context, request *workflowservice.CountWorkflowExecutionsRequest) (*workflowservice.CountWorkflowExecutionsResponse, error) { + panic("not implemented in the test environment") +} + +// DescribeTaskQueue implements Client. +func (t *testSuiteClientForNexusOperations) DescribeTaskQueue(ctx context.Context, taskqueue string, taskqueueType enums.TaskQueueType) (*workflowservice.DescribeTaskQueueResponse, error) { + panic("not implemented in the test environment") +} + +// DescribeTaskQueueEnhanced implements Client. +func (t *testSuiteClientForNexusOperations) DescribeTaskQueueEnhanced(ctx context.Context, options DescribeTaskQueueEnhancedOptions) (TaskQueueDescription, error) { + panic("unimplemented in the test environment") +} + +// DescribeWorkflowExecution implements Client. +func (t *testSuiteClientForNexusOperations) DescribeWorkflowExecution(ctx context.Context, workflowID string, runID string) (*workflowservice.DescribeWorkflowExecutionResponse, error) { + panic("not implemented in the test environment") +} + +// ExecuteWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) ExecuteWorkflow(ctx context.Context, options StartWorkflowOptions, workflow interface{}, args ...interface{}) (WorkflowRun, error) { + if set, ok := ctx.Value(IsWorkflowRunOpContextKey).(bool); !ok || !set { + panic("not implemented in the test environment") + } + wfType, input, err := getValidatedWorkflowFunction(workflow, args, t.env.dataConverter, t.env.GetRegistry()) + if err != nil { + return nil, fmt.Errorf("cannot validate workflow function: %w", err) + } + + run := &testEnvWorkflowRunForNexusOperations{} + doneCh := make(chan error) + + var callback *commonpb.Callback + + if len(options.callbacks) > 0 { + callback = options.callbacks[0] + } + + t.env.postCallback(func() { + t.env.executeChildWorkflowWithDelay(options.StartDelay, ExecuteWorkflowParams{ + // Not propagating Header as this client does not support context propagation. + WorkflowType: wfType, + Input: input, + WorkflowOptions: WorkflowOptions{ + WaitForCancellation: true, + Namespace: t.env.workflowInfo.Namespace, + TaskQueueName: t.env.workflowInfo.TaskQueueName, + WorkflowID: options.ID, + WorkflowExecutionTimeout: options.WorkflowExecutionTimeout, + WorkflowRunTimeout: options.WorkflowRunTimeout, + WorkflowTaskTimeout: options.WorkflowTaskTimeout, + DataConverter: t.env.dataConverter, + WorkflowIDReusePolicy: options.WorkflowIDReusePolicy, + ContextPropagators: t.env.contextPropagators, + SearchAttributes: options.SearchAttributes, + TypedSearchAttributes: options.TypedSearchAttributes, + ParentClosePolicy: enums.PARENT_CLOSE_POLICY_ABANDON, + Memo: options.Memo, + CronSchedule: options.CronSchedule, + RetryPolicy: convertToPBRetryPolicy(options.RetryPolicy), + }, + }, func(result *commonpb.Payloads, wfErr error) { + ncb := callback.GetNexus() + if ncb == nil { + return + } + seqStr := ncb.GetHeader()["operation-sequence"] + if seqStr == "" { + return + } + seq, err := strconv.ParseInt(seqStr, 10, 64) + if err != nil { + panic(fmt.Errorf("unexpected operation sequence in callback header: %s: %w", seqStr, err)) + } + + if wfErr != nil { + t.env.resolveNexusOperation(seq, nil, wfErr) + } else { + var payload *commonpb.Payload + if len(result.GetPayloads()) > 0 { + payload = result.Payloads[0] + } + t.env.resolveNexusOperation(seq, payload, nil) + } + }, func(r WorkflowExecution, err error) { + run.WorkflowExecution = r + doneCh <- err + }) + }, false) + err = <-doneCh + if err != nil { + return nil, err + } + return run, nil +} + +// GetSearchAttributes implements Client. +func (t *testSuiteClientForNexusOperations) GetSearchAttributes(ctx context.Context) (*workflowservice.GetSearchAttributesResponse, error) { + panic("not implemented in the test environment") +} + +// GetWorkerBuildIdCompatibility implements Client. +func (t *testSuiteClientForNexusOperations) GetWorkerBuildIdCompatibility(ctx context.Context, options *GetWorkerBuildIdCompatibilityOptions) (*WorkerBuildIDVersionSets, error) { + panic("not implemented in the test environment") +} + +// GetWorkerTaskReachability implements Client. +func (t *testSuiteClientForNexusOperations) GetWorkerTaskReachability(ctx context.Context, options *GetWorkerTaskReachabilityOptions) (*WorkerTaskReachability, error) { + panic("not implemented in the test environment") +} + +// GetWorkerVersioningRules implements Client. +func (t *testSuiteClientForNexusOperations) GetWorkerVersioningRules(ctx context.Context, options GetWorkerVersioningOptions) (*WorkerVersioningRules, error) { + panic("unimplemented in the test environment") +} + +// GetWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) GetWorkflow(ctx context.Context, workflowID string, runID string) WorkflowRun { + panic("not implemented in the test environment") +} + +// GetWorkflowHistory implements Client. +func (t *testSuiteClientForNexusOperations) GetWorkflowHistory(ctx context.Context, workflowID string, runID string, isLongPoll bool, filterType enums.HistoryEventFilterType) HistoryEventIterator { + panic("not implemented in the test environment") +} + +// GetWorkflowUpdateHandle implements Client. +func (t *testSuiteClientForNexusOperations) GetWorkflowUpdateHandle(GetWorkflowUpdateHandleOptions) WorkflowUpdateHandle { + panic("not implemented in the test environment") +} + +// ListArchivedWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) ListArchivedWorkflow(ctx context.Context, request *workflowservice.ListArchivedWorkflowExecutionsRequest) (*workflowservice.ListArchivedWorkflowExecutionsResponse, error) { + panic("not implemented in the test environment") +} + +// ListClosedWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) ListClosedWorkflow(ctx context.Context, request *workflowservice.ListClosedWorkflowExecutionsRequest) (*workflowservice.ListClosedWorkflowExecutionsResponse, error) { + panic("not implemented in the test environment") +} + +// ListOpenWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) ListOpenWorkflow(ctx context.Context, request *workflowservice.ListOpenWorkflowExecutionsRequest) (*workflowservice.ListOpenWorkflowExecutionsResponse, error) { + panic("not implemented in the test environment") +} + +// ListWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) ListWorkflow(ctx context.Context, request *workflowservice.ListWorkflowExecutionsRequest) (*workflowservice.ListWorkflowExecutionsResponse, error) { + panic("not implemented in the test environment") +} + +// OperatorService implements Client. +func (t *testSuiteClientForNexusOperations) OperatorService() operatorservice.OperatorServiceClient { + panic("not implemented in the test environment") +} + +// QueryWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) QueryWorkflow(ctx context.Context, workflowID string, runID string, queryType string, args ...interface{}) (converter.EncodedValue, error) { + panic("not implemented in the test environment") +} + +// QueryWorkflowWithOptions implements Client. +func (t *testSuiteClientForNexusOperations) QueryWorkflowWithOptions(ctx context.Context, request *QueryWorkflowWithOptionsRequest) (*QueryWorkflowWithOptionsResponse, error) { + panic("not implemented in the test environment") +} + +// RecordActivityHeartbeat implements Client. +func (t *testSuiteClientForNexusOperations) RecordActivityHeartbeat(ctx context.Context, taskToken []byte, details ...interface{}) error { + panic("not implemented in the test environment") +} + +// RecordActivityHeartbeatByID implements Client. +func (t *testSuiteClientForNexusOperations) RecordActivityHeartbeatByID(ctx context.Context, namespace string, workflowID string, runID string, activityID string, details ...interface{}) error { + panic("not implemented in the test environment") +} + +// ResetWorkflowExecution implements Client. +func (t *testSuiteClientForNexusOperations) ResetWorkflowExecution(ctx context.Context, request *workflowservice.ResetWorkflowExecutionRequest) (*workflowservice.ResetWorkflowExecutionResponse, error) { + panic("not implemented in the test environment") +} + +// ScanWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) ScanWorkflow(ctx context.Context, request *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error) { + panic("not implemented in the test environment") +} + +// ScheduleClient implements Client. +func (t *testSuiteClientForNexusOperations) ScheduleClient() ScheduleClient { + panic("not implemented in the test environment") +} + +// SignalWithStartWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) SignalWithStartWorkflow(ctx context.Context, workflowID string, signalName string, signalArg interface{}, options StartWorkflowOptions, workflow interface{}, workflowArgs ...interface{}) (WorkflowRun, error) { + panic("not implemented in the test environment") +} + +// SignalWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) SignalWorkflow(ctx context.Context, workflowID string, runID string, signalName string, arg interface{}) error { + panic("not implemented in the test environment") +} + +// TerminateWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) TerminateWorkflow(ctx context.Context, workflowID string, runID string, reason string, details ...interface{}) error { + panic("not implemented in the test environment") +} + +// UpdateWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) UpdateWorkflow(ctx context.Context, options UpdateWorkflowOptions) (WorkflowUpdateHandle, error) { + panic("unimplemented in the test environment") +} + +// UpdateWorkerBuildIdCompatibility implements Client. +func (t *testSuiteClientForNexusOperations) UpdateWorkerBuildIdCompatibility(ctx context.Context, options *UpdateWorkerBuildIdCompatibilityOptions) error { + panic("not implemented in the test environment") +} + +// UpdateWorkerVersioningRules implements Client. +func (t *testSuiteClientForNexusOperations) UpdateWorkerVersioningRules(ctx context.Context, options UpdateWorkerVersioningRulesOptions) (*WorkerVersioningRules, error) { + panic("unimplemented in the test environment") +} + +// WorkflowService implements Client. +func (t *testSuiteClientForNexusOperations) WorkflowService() workflowservice.WorkflowServiceClient { + panic("not implemented in the test environment") +} + +var _ Client = &testSuiteClientForNexusOperations{} + +// testEnvWorkflowRunForNexusOperations is a partial [WorkflowRun] implementation for the test workflow environment used +// to support basic Nexus functionality. +type testEnvWorkflowRunForNexusOperations struct { + WorkflowExecution +} + +// Get implements WorkflowRun. +func (t *testEnvWorkflowRunForNexusOperations) Get(ctx context.Context, valuePtr interface{}) error { + panic("not implemented in the test environment") +} + +// GetID implements WorkflowRun. +func (t *testEnvWorkflowRunForNexusOperations) GetID() string { + return t.ID +} + +// GetRunID implements WorkflowRun. +func (t *testEnvWorkflowRunForNexusOperations) GetRunID() string { + return t.RunID +} + +// GetWithOptions implements WorkflowRun. +func (t *testEnvWorkflowRunForNexusOperations) GetWithOptions(ctx context.Context, valuePtr interface{}, options WorkflowRunGetOptions) error { + panic("not implemented in the test environment") +} + +var _ WorkflowRun = &testEnvWorkflowRunForNexusOperations{} diff --git a/vendor/go.temporal.io/sdk/internal/protocol/registry.go b/vendor/go.temporal.io/sdk/internal/protocol/registry.go new file mode 100644 index 00000000000..acc68006701 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/protocol/registry.go @@ -0,0 +1,74 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package protocol + +import ( + "sync" + + protocolpb "go.temporal.io/api/protocol/v1" +) + +type ( + // Instance is the required interface for protocol objects. + Instance interface { + HandleMessage(*protocolpb.Message) error + HasCompleted() bool + } + + // Registry stores running protocols. + Registry struct { + mut sync.Mutex + instances map[string]Instance + } +) + +func NewRegistry() *Registry { + return &Registry{instances: map[string]Instance{}} +} + +// FindOrAdd looks up an existing protocol by instance ID or constructs a new +// one and registers it under the instance ID indicated. +func (r *Registry) FindOrAdd(instID string, ctor func() Instance) Instance { + r.mut.Lock() + defer r.mut.Unlock() + p, ok := r.instances[instID] + if !ok { + p = ctor() + r.instances[instID] = p + } + return p +} + +// ClearCompleted walks the registered protocols and removes those that have +// completed. +func (r *Registry) ClearCompleted() { + r.mut.Lock() + defer r.mut.Unlock() + for instID, inst := range r.instances { + if inst.HasCompleted() { + delete(r.instances, instID) + } + } +} diff --git a/vendor/go.temporal.io/sdk/internal/protocol/util.go b/vendor/go.temporal.io/sdk/internal/protocol/util.go new file mode 100644 index 00000000000..f117771edf5 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/protocol/util.go @@ -0,0 +1,59 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package protocol + +import ( + "errors" + "strings" + + protocolpb "go.temporal.io/api/protocol/v1" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/anypb" +) + +var ErrProtoNameNotFound = errors.New("protocol name not found") + +// NameFromMessage extracts the name of the protocol to which the supplied +// message belongs. +func NameFromMessage(msg *protocolpb.Message) (string, error) { + bodyType := string(msg.GetBody().MessageName()) + if bodyType == "" { + return "", ErrProtoNameNotFound + } + + if lastDot := strings.LastIndex(bodyType, "."); lastDot > -1 { + bodyType = bodyType[0:lastDot] + } + return bodyType, nil +} + +// MustMarshalAny serializes a protobuf message into an Any or panics. +func MustMarshalAny(msg proto.Message) *anypb.Any { + result, err := anypb.New(msg) + if err != nil { + panic(err) + } + return result +} diff --git a/vendor/go.temporal.io/sdk/internal/schedule_client.go b/vendor/go.temporal.io/sdk/internal/schedule_client.go new file mode 100644 index 00000000000..c6ea706b2b4 --- /dev/null +++ b/vendor/go.temporal.io/sdk/internal/schedule_client.go @@ -0,0 +1,665 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "time" + + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" +) + +type ( + // ScheduleRange represents a set of integer values, used to match fields of a calendar + // time in StructuredCalendarSpec. If end < start, then end is interpreted as + // equal to start. This means you can use a Range with start set to a value, and + // end and step unset (defaulting to 0) to represent a single value. + ScheduleRange struct { + // Start of the range (inclusive) + Start int + + // End of the range (inclusive) + // Optional: defaulted to Start + End int + + // Step to be take between each value + // Optional: defaulted to 1 + Step int + } + + // ScheduleCalendarSpec is an event specification relative to the calendar, similar to a traditional cron specification. + // A timestamp matches if at least one range of each field matches the + // corresponding fields of the timestamp, except for year: if year is missing, + // that means all years match. For all fields besides year, at least one Range must be present to match anything. + ScheduleCalendarSpec struct { + // Second range to match (0-59). + // + // default: matches 0 + Second []ScheduleRange + + // Minute range to match (0-59). + // + // default: matches 0 + Minute []ScheduleRange + + // Hour range to match (0-23). + // + // default: matches 0 + Hour []ScheduleRange + + // DayOfMonth range to match (1-31) + // + // default: matches all days + DayOfMonth []ScheduleRange + + // Month range to match (1-12) + // + // default: matches all months + Month []ScheduleRange + + // Year range to match. + // + // default: empty that matches all years + Year []ScheduleRange + + // DayOfWeek range to match (0-6; 0 is Sunday) + // + // default: matches all days of the week + DayOfWeek []ScheduleRange + + // Comment - Description of the intention of this schedule. + Comment string + } + + // ScheduleBackfill desribes a time periods and policy and takes Actions as if that time passed by right now, all at once. + ScheduleBackfill struct { + // Start - start of the range to evaluate schedule in. + Start time.Time + + // End - end of the range to evaluate schedule in. + End time.Time + + // Overlap - Override the Overlap Policy for this request. + Overlap enumspb.ScheduleOverlapPolicy + } + + // ScheduleIntervalSpec - matches times that can be expressed as: + // + // Epoch + (n * every) + offset + // + // where n is all integers ≥ 0. + // + // For example, an `every` of 1 hour with `offset` of zero would match every hour, on the hour. The same `every` but an `offset` + // of 19 minutes would match every `xx:19:00`. An `every` of 28 days with `offset` zero would match `2022-02-17T00:00:00Z` + // (among other times). The same `every` with `offset` of 3 days, 5 hours, and 23 minutes would match `2022-02-20T05:23:00Z` + // instead. + ScheduleIntervalSpec struct { + // Every - describes the period to repeat the interval. + Every time.Duration + + // Offset - is a fixed offset added to the intervals period. + // Optional: Defaulted to 0 + Offset time.Duration + } + + // ScheduleSpec is a complete description of a set of absolute times (possibly infinite) that a action should occur at. + // The times are the union of Calendars, Intervals, and CronExpressions, minus the Skip times. These times + // never change, except that the definition of a time zone can change over time (most commonly, when daylight saving + // time policy changes for an area). To create a totally self-contained ScheduleSpec, use UTC. + ScheduleSpec struct { + // Calendars - Calendar-based specifications of times + Calendars []ScheduleCalendarSpec + + // Intervals - Interval-based specifications of times. + Intervals []ScheduleIntervalSpec + + // CronExpressions - CronExpressions-based specifications of times. CronExpressions is provided for easy migration from legacy Cron Workflows. For new + // use cases, we recommend using ScheduleSpec.Calendars or ScheduleSpec.Intervals for readability and maintainability. Once a schedule is created all + // expressions in CronExpressions will be translated to ScheduleSpec.Calendars on the server. + // + // For example, `0 12 * * MON-WED,FRI` is every M/Tu/W/F at noon, and is equivalent to this ScheduleCalendarSpec: + // + // client.ScheduleCalendarSpec{ + // Second: []ScheduleRange{{}}, + // Minute: []ScheduleRanges{{}}, + // Hour: []ScheduleRange{{ + // Start: 12, + // }}, + // DayOfMonth: []ScheduleRange{ + // { + // Start: 1, + // End: 31, + // }, + // }, + // Month: []ScheduleRange{ + // { + // Start: 1, + // End: 12, + // }, + // }, + // DayOfWeek: []ScheduleRange{ + // { + // Start: 1, + // End: 3, + // }, + // { + // Start: 5, + // }, + // }, + // } + // + // + // The string can have 5, 6, or 7 fields, separated by spaces, and they are interpreted in the + // same way as a ScheduleCalendarSpec: + // - 5 fields: Minute, Hour, DayOfMonth, Month, DayOfWeek + // - 6 fields: Minute, Hour, DayOfMonth, Month, DayOfWeek, Year + // - 7 fields: Second, Minute, Hour, DayOfMonth, Month, DayOfWeek, Year + // + // Notes: + // - If Year is not given, it defaults to *. + // - If Second is not given, it defaults to 0. + // - Shorthands @yearly, @monthly, @weekly, @daily, and @hourly are also + // accepted instead of the 5-7 time fields. + // - @every [/] is accepted and gets compiled into an + // IntervalSpec instead. and should be a decimal integer + // with a unit suffix s, m, h, or d. + // - Optionally, the string can be preceded by CRON_TZ=